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/05 15:39:42 UTC

[lucene] 01/03: LUCENE-10291: Only read/write postings when there is at least one indexed field (#539)

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 738247e78d1d5ff22f3755d0ceca8ad99d4f69f4
Author: Yannick Welsch <ya...@welsch.lu>
AuthorDate: Wed Jan 5 16:28:00 2022 +0100

    LUCENE-10291: Only read/write postings when there is at least one indexed field (#539)
---
 .../org/apache/lucene/codecs/FieldsConsumer.java   |   8 +-
 .../lucene/codecs/perfield/PerFieldMergeState.java |  14 +-
 .../codecs/perfield/PerFieldPostingsFormat.java    |   2 +
 .../java/org/apache/lucene/index/CheckIndex.java   |  19 ++-
 .../java/org/apache/lucene/index/CodecReader.java  |  10 +-
 .../java/org/apache/lucene/index/FieldInfos.java   |   9 ++
 .../apache/lucene/index/FreqProxTermsWriter.java   |   5 +
 .../java/org/apache/lucene/index/MergeState.java   |   6 +-
 .../org/apache/lucene/index/SegmentCommitInfo.java |   4 +-
 .../apache/lucene/index/SegmentCoreReaders.java    |  12 +-
 .../org/apache/lucene/index/SegmentMerger.java     |   6 +-
 .../services/org.apache.lucene.codecs.Codec        |  17 +++
 .../org/apache/lucene/codecs/TestMinimalCodec.java | 162 +++++++++++++++++++++
 13 files changed, 257 insertions(+), 17 deletions(-)

diff --git a/lucene/core/src/java/org/apache/lucene/codecs/FieldsConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/FieldsConsumer.java
index 0659827..cf0f796 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/FieldsConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/FieldsConsumer.java
@@ -79,9 +79,11 @@ public abstract class FieldsConsumer implements Closeable {
       final FieldsProducer f = mergeState.fieldsProducers[readerIndex];
 
       final int maxDoc = mergeState.maxDocs[readerIndex];
-      f.checkIntegrity();
-      slices.add(new ReaderSlice(docBase, maxDoc, readerIndex));
-      fields.add(f);
+      if (f != null) {
+        f.checkIntegrity();
+        slices.add(new ReaderSlice(docBase, maxDoc, readerIndex));
+        fields.add(f);
+      }
       docBase += maxDoc;
     }
 
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldMergeState.java b/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldMergeState.java
index 47fe0df..c340448 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldMergeState.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldMergeState.java
@@ -79,7 +79,10 @@ final class PerFieldMergeState {
       in.fieldInfos[i] = new FilterFieldInfos(orgFieldInfos[i], fields);
     }
     for (int i = 0; i < orgFieldsProducers.length; i++) {
-      in.fieldsProducers[i] = new FilterFieldsProducer(orgFieldsProducers[i], fields);
+      in.fieldsProducers[i] =
+          orgFieldsProducers[i] == null
+              ? null
+              : new FilterFieldsProducer(orgFieldsProducers[i], fields);
     }
     return in;
   }
@@ -103,6 +106,7 @@ final class PerFieldMergeState {
     // Copy of the private fields from FieldInfos
     // Renamed so as to be less confusing about which fields we're referring to
     private final boolean filteredHasVectors;
+    private final boolean filteredHasPostings;
     private final boolean filteredHasProx;
     private final boolean filteredHasPayloads;
     private final boolean filteredHasOffsets;
@@ -116,6 +120,7 @@ final class PerFieldMergeState {
       super(toArray(src));
 
       boolean hasVectors = false;
+      boolean hasPostings = false;
       boolean hasProx = false;
       boolean hasPayloads = false;
       boolean hasOffsets = false;
@@ -130,6 +135,7 @@ final class PerFieldMergeState {
         if (this.filteredNames.contains(fi.name)) {
           this.filtered.add(fi);
           hasVectors |= fi.hasVectors();
+          hasPostings |= fi.getIndexOptions() != IndexOptions.NONE;
           hasProx |= fi.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
           hasFreq |= fi.getIndexOptions() != IndexOptions.DOCS;
           hasOffsets |=
@@ -143,6 +149,7 @@ final class PerFieldMergeState {
       }
 
       this.filteredHasVectors = hasVectors;
+      this.filteredHasPostings = hasPostings;
       this.filteredHasProx = hasProx;
       this.filteredHasPayloads = hasPayloads;
       this.filteredHasOffsets = hasOffsets;
@@ -172,6 +179,11 @@ final class PerFieldMergeState {
     }
 
     @Override
+    public boolean hasPostings() {
+      return filteredHasPostings;
+    }
+
+    @Override
     public boolean hasProx() {
       return filteredHasProx;
     }
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java
index d183f62..6468897 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java
@@ -27,6 +27,7 @@ import java.util.IdentityHashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.ServiceLoader;
 import java.util.Set;
 import java.util.TreeMap;
@@ -185,6 +186,7 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
               new MergedIterator<>(
                   true,
                   Arrays.stream(mergeState.fieldsProducers)
+                      .filter(Objects::nonNull)
                       .map(FieldsProducer::iterator)
                       .toArray(Iterator[]::new));
       Map<PostingsFormat, FieldsGroup> formatToGroups = buildFieldsGroupMapping(indexedFieldNames);
diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
index de5141a..f699937 100644
--- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
+++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
@@ -44,6 +44,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.function.Supplier;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.codecs.NormsProducer;
 import org.apache.lucene.codecs.PointsReader;
 import org.apache.lucene.codecs.PostingsFormat;
@@ -2407,7 +2408,12 @@ public final class CheckIndex implements Closeable {
         infoStream.print("    test: terms, freq, prox...");
       }
 
-      final Fields fields = reader.getPostingsReader().getMergeInstance();
+      FieldsProducer fields = reader.getPostingsReader();
+      if (fields != null) {
+        fields = fields.getMergeInstance();
+      } else {
+        return new Status.TermIndexStatus();
+      }
       final FieldInfos fieldInfos = reader.getFieldInfos();
       NormsProducer normsProducer = reader.getNormsReader();
       if (normsProducer != null) {
@@ -3540,10 +3546,13 @@ public final class CheckIndex implements Closeable {
 
       final Bits liveDocs = reader.getLiveDocs();
 
-      final Fields postingsFields;
+      FieldsProducer postingsFields;
       // TODO: testTermsIndex
       if (doSlowChecks) {
-        postingsFields = reader.getPostingsReader().getMergeInstance();
+        postingsFields = reader.getPostingsReader();
+        if (postingsFields != null) {
+          postingsFields = postingsFields.getMergeInstance();
+        }
       } else {
         postingsFields = null;
       }
@@ -3597,6 +3606,10 @@ public final class CheckIndex implements Closeable {
                 final boolean postingsHasPayload = fieldInfo.hasPayloads();
                 final boolean vectorsHasPayload = terms.hasPayloads();
 
+                if (postingsFields == null) {
+                  throw new CheckIndexException(
+                      "vector field=" + field + " does not exist in postings; doc=" + j);
+                }
                 Terms postingsTerms = postingsFields.terms(field);
                 if (postingsTerms == null) {
                   throw new CheckIndexException(
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 f78f126..064cdbf 100644
--- a/lucene/core/src/java/org/apache/lucene/index/CodecReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/CodecReader.java
@@ -106,10 +106,14 @@ 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) {
+      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 getPostingsReader().terms(field);
+    return fieldsProducer.terms(field);
   }
 
   // returns the FieldInfo that corresponds to the given field and type, or
@@ -241,7 +245,9 @@ public abstract class CodecReader extends LeafReader {
     ensureOpen();
 
     // terms/postings
-    getPostingsReader().checkIntegrity();
+    if (getPostingsReader() != null) {
+      getPostingsReader().checkIntegrity();
+    }
 
     // norms
     if (getNormsReader() != null) {
diff --git a/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java b/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
index 44fb021..76cf9bb 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
@@ -48,6 +48,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
   public static final FieldInfos EMPTY = new FieldInfos(new FieldInfo[0]);
 
   private final boolean hasFreq;
+  private final boolean hasPostings;
   private final boolean hasProx;
   private final boolean hasPayloads;
   private final boolean hasOffsets;
@@ -67,6 +68,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
   /** Constructs a new FieldInfos from an array of FieldInfo objects */
   public FieldInfos(FieldInfo[] infos) {
     boolean hasVectors = false;
+    boolean hasPostings = false;
     boolean hasProx = false;
     boolean hasPayloads = false;
     boolean hasOffsets = false;
@@ -112,6 +114,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
       }
 
       hasVectors |= info.hasVectors();
+      hasPostings |= info.getIndexOptions() != IndexOptions.NONE;
       hasProx |= info.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
       hasFreq |= info.getIndexOptions() != IndexOptions.DOCS;
       hasOffsets |=
@@ -132,6 +135,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
     }
 
     this.hasVectors = hasVectors;
+    this.hasPostings = hasPostings;
     this.hasProx = hasProx;
     this.hasPayloads = hasPayloads;
     this.hasOffsets = hasOffsets;
@@ -200,6 +204,11 @@ public class FieldInfos implements Iterable<FieldInfo> {
     return hasFreq;
   }
 
+  /** Returns true if any fields have postings */
+  public boolean hasPostings() {
+    return hasPostings;
+  }
+
   /** Returns true if any fields have positions */
   public boolean hasProx() {
     return hasProx;
diff --git a/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java b/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
index e417549..ca2775c 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
@@ -99,6 +99,11 @@ final class FreqProxTermsWriter extends TermsHash {
       }
     }
 
+    if (!state.fieldInfos.hasPostings()) {
+      assert allFields.isEmpty();
+      return;
+    }
+
     // Sort by field name
     CollectionUtil.introSort(allFields);
 
diff --git a/lucene/core/src/java/org/apache/lucene/index/MergeState.java b/lucene/core/src/java/org/apache/lucene/index/MergeState.java
index 6bc7e76..faea6cb 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MergeState.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MergeState.java
@@ -132,7 +132,11 @@ public class MergeState {
         termVectorsReaders[i] = termVectorsReaders[i].getMergeInstance();
       }
 
-      fieldsProducers[i] = reader.getPostingsReader().getMergeInstance();
+      fieldsProducers[i] = reader.getPostingsReader();
+      if (fieldsProducers[i] != null) {
+        fieldsProducers[i] = fieldsProducers[i].getMergeInstance();
+      }
+
       pointsReaders[i] = reader.getPointsReader();
       if (pointsReaders[i] != null) {
         pointsReaders[i] = pointsReaders[i].getMergeInstance();
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java b/lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java
index b8bb1d7..9357cf1 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java
@@ -244,7 +244,9 @@ public class SegmentCommitInfo {
     // updates) and then maybe even be able to remove LiveDocsFormat.files().
 
     // Must separately add any live docs files:
-    info.getCodec().liveDocsFormat().files(this, files);
+    if (hasDeletions()) {
+      info.getCodec().liveDocsFormat().files(this, files);
+    }
 
     // must separately add any field updates files
     for (Set<String> updatefiles : dvUpdatesFiles.values()) {
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java b/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
index 68de0f2..df733b1 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
@@ -112,10 +112,14 @@ final class SegmentCoreReaders {
 
       final SegmentReadState segmentReadState =
           new SegmentReadState(cfsDir, si.info, coreFieldInfos, context);
-      final PostingsFormat format = codec.postingsFormat();
-      // Ask codec for its Fields
-      fields = format.fieldsProducer(segmentReadState);
-      assert fields != null;
+      if (coreFieldInfos.hasPostings()) {
+        final PostingsFormat format = codec.postingsFormat();
+        // Ask codec for its Fields
+        fields = format.fieldsProducer(segmentReadState);
+        assert fields != null;
+      } else {
+        fields = null;
+      }
       // ask codec for its Norms:
       // TODO: since we don't write any norms file if there are no norms,
       // kinda jaky to assume the codec handles the case of no norms file at all gracefully?!
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java b/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
index 689b772..805b5e5 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
@@ -203,8 +203,10 @@ final class SegmentMerger {
         // Use the merge instance in order to reuse the same IndexInput for all terms
         normsMergeInstance = norms.getMergeInstance();
       }
-      try (FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(segmentWriteState)) {
-        consumer.merge(mergeState, normsMergeInstance);
+      if (mergeState.mergeFieldInfos.hasPostings()) {
+        try (FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(segmentWriteState)) {
+          consumer.merge(mergeState, normsMergeInstance);
+        }
       }
     }
   }
diff --git a/lucene/core/src/test/META-INF/services/org.apache.lucene.codecs.Codec b/lucene/core/src/test/META-INF/services/org.apache.lucene.codecs.Codec
new file mode 100644
index 0000000..8c7c0df
--- /dev/null
+++ b/lucene/core/src/test/META-INF/services/org.apache.lucene.codecs.Codec
@@ -0,0 +1,17 @@
+#  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.
+
+org.apache.lucene.codecs.TestMinimalCodec$MinimalCodec
+org.apache.lucene.codecs.TestMinimalCodec$MinimalCompoundCodec
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/TestMinimalCodec.java b/lucene/core/src/test/org/apache/lucene/codecs/TestMinimalCodec.java
new file mode 100644
index 0000000..c3c4edb
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/codecs/TestMinimalCodec.java
@@ -0,0 +1,162 @@
+/*
+ * 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;
+
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomBoolean;
+
+import java.io.IOException;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.tests.analysis.MockAnalyzer;
+import org.apache.lucene.tests.store.BaseDirectoryWrapper;
+import org.apache.lucene.tests.util.LuceneTestCase;
+import org.apache.lucene.tests.util.TestUtil;
+
+/**
+ * Tests to ensure that {@link Codec}s won't need to implement all formats in case where only a
+ * small subset of Lucene's functionality is used.
+ */
+public class TestMinimalCodec extends LuceneTestCase {
+
+  public void testMinimalCodec() throws IOException {
+    runMinimalCodecTest(false);
+  }
+
+  public void testMinimalCompoundCodec() throws IOException {
+    runMinimalCodecTest(true);
+  }
+
+  private void runMinimalCodecTest(boolean useCompoundFile) throws IOException {
+    try (BaseDirectoryWrapper dir = newDirectory()) {
+      IndexWriterConfig writerConfig =
+          newIndexWriterConfig(new MockAnalyzer(random()))
+              .setCodec(useCompoundFile ? new MinimalCompoundCodec() : new MinimalCodec())
+              .setUseCompoundFile(useCompoundFile);
+      if (!useCompoundFile) {
+        writerConfig.getMergePolicy().setNoCFSRatio(0.0);
+        writerConfig.getMergePolicy().setMaxCFSSegmentSizeMB(Double.POSITIVE_INFINITY);
+      }
+
+      try (IndexWriter writer = new IndexWriter(dir, writerConfig)) {
+        writer.addDocument(basicDocument());
+        writer.flush();
+        // create second segment
+        writer.addDocument(basicDocument());
+        writer.forceMerge(1); // test merges
+        if (randomBoolean()) {
+          writer.commit();
+        }
+
+        try (DirectoryReader reader = DirectoryReader.open(writer)) {
+          assertEquals(2, reader.numDocs());
+        }
+      }
+    }
+  }
+
+  /** returns a basic document with no indexed fields */
+  private static Document basicDocument() {
+    return new Document();
+  }
+
+  /** Minimal codec implementation for working with the most basic documents */
+  public static class MinimalCodec extends Codec {
+
+    protected final Codec wrappedCodec = TestUtil.getDefaultCodec();
+
+    public MinimalCodec() {
+      this("MinimalCodec");
+    }
+
+    protected MinimalCodec(String name) {
+      super(name);
+    }
+
+    @Override
+    public FieldInfosFormat fieldInfosFormat() {
+      return wrappedCodec.fieldInfosFormat();
+    }
+
+    @Override
+    public SegmentInfoFormat segmentInfoFormat() {
+      return wrappedCodec.segmentInfoFormat();
+    }
+
+    @Override
+    public CompoundFormat compoundFormat() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public LiveDocsFormat liveDocsFormat() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public StoredFieldsFormat storedFieldsFormat() {
+      // TODO: avoid calling this when no stored fields are written or read
+      return wrappedCodec.storedFieldsFormat();
+    }
+
+    @Override
+    public PostingsFormat postingsFormat() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public DocValuesFormat docValuesFormat() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public TermVectorsFormat termVectorsFormat() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public NormsFormat normsFormat() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public PointsFormat pointsFormat() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public KnnVectorsFormat knnVectorsFormat() {
+      throw new UnsupportedOperationException();
+    }
+  }
+
+  /**
+   * Minimal codec implementation for working with the most basic documents, supporting compound
+   * formats
+   */
+  public static class MinimalCompoundCodec extends MinimalCodec {
+    public MinimalCompoundCodec() {
+      super("MinimalCompoundCodec");
+    }
+
+    @Override
+    public CompoundFormat compoundFormat() {
+      return wrappedCodec.compoundFormat();
+    }
+  }
+}