You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2021/03/11 13:46:14 UTC

[GitHub] [lucene] mayya-sharipova opened a new pull request #11: LUCENE-9334 Consistency of field data structures

mayya-sharipova opened a new pull request #11:
URL: https://github.com/apache/lucene/pull/11


   Require consistency between data-structures on a per-field basis
   
   A field must be indexed with the same index options and data-structures across
   all documents within a segment. Thus, for example, it is not allowed to have
   one document in a segment where a certain field is indexed with doc values
   and points, and another document where the same field is indexed only with
   points. But it is allowed for a document not to have a certain field at all.
   
   This requirement is ensured only per segment basis, thus it is possible
   to have a index where two segments have different data structures
   for the same field.
   
   This patch also corrects tests where fields were indexed inconsistently
   with different data structures across documents:
   
   Remove TestDocValuesIndexing::testDocsWithField, as it doesn't
   test anything special, what it was originally created to test


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r596373649



##########
File path: lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java
##########
@@ -149,9 +149,6 @@ boolean start(IndexableField field, boolean first) {
       doVectors = field.fieldType().storeTermVectors();
 
       if (doVectors) {
-
-        termsWriter.hasVectors = true;

Review comment:
       We do this on the FieldInfo creation, in the `IndexingChain.PerField.setInvertState`: 
   ```java
   if (fieldInfo.hasVectors()) {
     termVectorsWriter.setHasVectors();
   }
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jpountz commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r605809392



##########
File path: lucene/core/src/test/org/apache/lucene/document/TestPerFieldConsistency.java
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.document;
+
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomDouble;
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomFloat;
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomInt;
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomIntBetween;
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomLong;
+
+import com.carrotsearch.randomizedtesting.generators.RandomPicks;
+import java.io.IOException;
+import java.util.Random;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestPerFieldConsistency extends LuceneTestCase {
+
+  private static Field randomIndexedField(Random random, String fieldName) {
+    FieldType fieldType = new FieldType();
+    IndexOptions indexOptions = RandomPicks.randomFrom(random, IndexOptions.values());
+    while (indexOptions == IndexOptions.NONE) {
+      indexOptions = RandomPicks.randomFrom(random, IndexOptions.values());
+    }
+    fieldType.setIndexOptions(indexOptions);
+    fieldType.setStoreTermVectors(random.nextBoolean());
+    if (fieldType.storeTermVectors()) {
+      fieldType.setStoreTermVectorPositions(random.nextBoolean());
+      if (fieldType.storeTermVectorPositions()) {
+        fieldType.setStoreTermVectorPayloads(random.nextBoolean());
+        fieldType.setStoreTermVectorOffsets(random.nextBoolean());
+      }
+    }
+    fieldType.setOmitNorms(random.nextBoolean());
+    fieldType.setStored(random.nextBoolean());
+    fieldType.freeze();
+
+    return new Field(fieldName, "randomValue", fieldType);
+  }
+
+  private static Field randomPointField(Random random, String fieldName) {
+    switch (random.nextInt(4)) {
+      case 0:
+        return new LongPoint(fieldName, randomLong());
+      case 1:
+        return new IntPoint(fieldName, randomInt());
+      case 2:
+        return new DoublePoint(fieldName, randomDouble());
+      default:
+        return new FloatPoint(fieldName, randomFloat());
+    }
+  }
+
+  private static Field randomDocValuesField(Random random, String fieldName) {
+    switch (random.nextInt(4)) {
+      case 0:
+        return new BinaryDocValuesField(fieldName, new BytesRef("randomValue"));
+      case 1:
+        return new NumericDocValuesField(fieldName, randomLong());
+      case 2:
+        return new DoubleDocValuesField(fieldName, randomDouble());
+      default:
+        return new SortedSetDocValuesField(fieldName, new BytesRef("randomValue"));
+    }
+  }
+
+  private static Field randomVectorField(Random random, String fieldName) {
+    VectorValues.SearchStrategy searchStrategy =
+        RandomPicks.randomFrom(random, VectorValues.SearchStrategy.values());
+    while (searchStrategy == VectorValues.SearchStrategy.NONE) {
+      searchStrategy = RandomPicks.randomFrom(random, VectorValues.SearchStrategy.values());
+    }
+    float[] values = new float[randomIntBetween(1, 10)];
+    for (int i = 0; i < values.length; i++) {
+      values[i] = randomFloat();
+    }
+    return new VectorField(fieldName, values, searchStrategy);
+  }
+
+  private static Field[] randomFieldsWithTheSameName(String fieldName) {
+    final Field textField = randomIndexedField(random(), fieldName);
+    final Field docValuesField = randomDocValuesField(random(), fieldName);
+    final Field pointField = randomPointField(random(), fieldName);
+    final Field vectorField = randomVectorField(random(), fieldName);
+    return new Field[] {textField, docValuesField, pointField, vectorField};
+  }
+
+  public void testDocWithMissingSchemaOptionsThrowsError() throws IOException {
+    try (Directory dir = newDirectory();
+        IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig()); ) {
+      final Field[] fields = randomFieldsWithTheSameName("myfield");
+      final Document doc0 = new Document();
+      for (Field field : fields) {
+        doc0.add(field);
+      }
+      writer.addDocument(doc0);
+
+      // the same segment: indexing a doc with a missing field throws error
+      int missingFieldIdx = randomIntBetween(0, fields.length - 1);

Review comment:
       likewise here, could we test all combinations everytime we run the test?

##########
File path: lucene/core/src/test/org/apache/lucene/document/TestPerFieldConsistency.java
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.document;
+
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomDouble;
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomFloat;
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomInt;
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomIntBetween;
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomLong;
+
+import com.carrotsearch.randomizedtesting.generators.RandomPicks;
+import java.io.IOException;
+import java.util.Random;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestPerFieldConsistency extends LuceneTestCase {
+
+  private static Field randomIndexedField(Random random, String fieldName) {
+    FieldType fieldType = new FieldType();
+    IndexOptions indexOptions = RandomPicks.randomFrom(random, IndexOptions.values());
+    while (indexOptions == IndexOptions.NONE) {
+      indexOptions = RandomPicks.randomFrom(random, IndexOptions.values());
+    }
+    fieldType.setIndexOptions(indexOptions);
+    fieldType.setStoreTermVectors(random.nextBoolean());
+    if (fieldType.storeTermVectors()) {
+      fieldType.setStoreTermVectorPositions(random.nextBoolean());
+      if (fieldType.storeTermVectorPositions()) {
+        fieldType.setStoreTermVectorPayloads(random.nextBoolean());
+        fieldType.setStoreTermVectorOffsets(random.nextBoolean());
+      }
+    }
+    fieldType.setOmitNorms(random.nextBoolean());
+    fieldType.setStored(random.nextBoolean());
+    fieldType.freeze();
+
+    return new Field(fieldName, "randomValue", fieldType);
+  }
+
+  private static Field randomPointField(Random random, String fieldName) {
+    switch (random.nextInt(4)) {
+      case 0:
+        return new LongPoint(fieldName, randomLong());
+      case 1:
+        return new IntPoint(fieldName, randomInt());
+      case 2:
+        return new DoublePoint(fieldName, randomDouble());
+      default:
+        return new FloatPoint(fieldName, randomFloat());
+    }
+  }
+
+  private static Field randomDocValuesField(Random random, String fieldName) {
+    switch (random.nextInt(4)) {
+      case 0:
+        return new BinaryDocValuesField(fieldName, new BytesRef("randomValue"));
+      case 1:
+        return new NumericDocValuesField(fieldName, randomLong());
+      case 2:
+        return new DoubleDocValuesField(fieldName, randomDouble());
+      default:
+        return new SortedSetDocValuesField(fieldName, new BytesRef("randomValue"));
+    }
+  }
+
+  private static Field randomVectorField(Random random, String fieldName) {
+    VectorValues.SearchStrategy searchStrategy =
+        RandomPicks.randomFrom(random, VectorValues.SearchStrategy.values());
+    while (searchStrategy == VectorValues.SearchStrategy.NONE) {
+      searchStrategy = RandomPicks.randomFrom(random, VectorValues.SearchStrategy.values());
+    }
+    float[] values = new float[randomIntBetween(1, 10)];
+    for (int i = 0; i < values.length; i++) {
+      values[i] = randomFloat();
+    }
+    return new VectorField(fieldName, values, searchStrategy);
+  }
+
+  private static Field[] randomFieldsWithTheSameName(String fieldName) {
+    final Field textField = randomIndexedField(random(), fieldName);
+    final Field docValuesField = randomDocValuesField(random(), fieldName);
+    final Field pointField = randomPointField(random(), fieldName);
+    final Field vectorField = randomVectorField(random(), fieldName);
+    return new Field[] {textField, docValuesField, pointField, vectorField};
+  }
+
+  public void testDocWithMissingSchemaOptionsThrowsError() throws IOException {
+    try (Directory dir = newDirectory();
+        IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig()); ) {
+      final Field[] fields = randomFieldsWithTheSameName("myfield");
+      final Document doc0 = new Document();
+      for (Field field : fields) {
+        doc0.add(field);
+      }
+      writer.addDocument(doc0);
+
+      // the same segment: indexing a doc with a missing field throws error
+      int missingFieldIdx = randomIntBetween(0, fields.length - 1);
+      final Document doc1 = new Document();
+      for (int i = 0; i < fields.length; i++) {
+        if (i != missingFieldIdx) {
+          doc1.add(fields[i]);
+        }
+      }
+      IllegalArgumentException exception =
+          expectThrows(IllegalArgumentException.class, () -> writer.addDocument(doc1));
+      String expectedErrMsg =
+          "Inconsistency of field data structures across documents for field [myfield] of doc [1].";
+      assertEquals(expectedErrMsg, exception.getMessage());
+
+      writer.flush();
+      try (IndexReader reader = DirectoryReader.open(writer)) {
+        LeafReader lr1 = reader.leaves().get(0).reader();
+        assertEquals(1, lr1.numDocs());
+        assertEquals(1, lr1.numDeletedDocs());
+      }
+
+      // diff segment, same index: indexing a doc with a missing field throws error
+      exception = expectThrows(IllegalArgumentException.class, () -> writer.addDocument(doc1));
+      assertTrue(exception.getMessage().contains("cannot change field \"myfield\" from "));
+
+      writer.addDocument(doc0); // add document with correct data structures
+
+      writer.flush();
+      try (IndexReader reader = DirectoryReader.open(writer)) {
+        LeafReader lr2 = reader.leaves().get(1).reader();
+        assertEquals(1, lr2.numDocs());
+        assertEquals(1, lr2.numDeletedDocs());
+      }
+    }
+  }
+
+  public void testDocWithExtraIndexingOptionsThrowsError() throws IOException {
+    try (Directory dir = newDirectory();
+        IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig())) {
+      final Field[] fields = randomFieldsWithTheSameName("myfield");
+      final Document doc0 = new Document();
+      int existingFieldIdx = randomIntBetween(0, fields.length - 1);
+      doc0.add(fields[existingFieldIdx]);
+      writer.addDocument(doc0);
+
+      // the same segment: indexing a field with extra field indexing options returns error
+      int extraFieldIndex = randomIntBetween(0, fields.length - 1);
+      while (extraFieldIndex == existingFieldIdx) {
+        extraFieldIndex = randomIntBetween(0, fields.length - 1);
+      }

Review comment:
       Could you rewrite this test so that we always test all combinations instead of testing just one? E.g. we could extract this to a `doTestDocWithExtraIndexingOptionsThrowsError(FieldType existing, FieldType extra)` and then one test for every combination?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r598287479



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -130,127 +167,252 @@ public boolean checkConsistency() {
       }
     }
 
-    if (pointDimensionCount < 0) {
+    if (docValuesType == null) {
+      throw new IllegalStateException("DocValuesType must not be null (field: '" + name + "')");

Review comment:
       Addressed in 6dc9f27 and 55fa8c8




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova edited a comment on pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova edited a comment on pull request #11:
URL: https://github.com/apache/lucene/pull/11#issuecomment-805296177


   I've run indexing benchmarking using [luceneutil](https://github.com/mikemccand/luceneutil). And here are the results:
   
   - indexing time in ms
   - baseline: master branch
   - candidate: this PR
   
   
   | Dataset        | Baseline  | Candidate | Difference |
   | :---           |     ---:  |      ---: |      ---:  |
   | wikimedium500k | 98387     | 106189    | 7.9%       |
   | wikimedium1m   | 174246    | 177075    | 1.6%       |
   | wikimedium10m  | 1356184   | 1359149   | 0.2%       |
   
   
   [wikimedium1m profiler](https://gist.github.com/mayya-sharipova/b4c8f47165a4bde8d2625487d2132319)
   [wikimedium10m profiler](https://gist.github.com/mayya-sharipova/68cf6d543863029777ad3028c662ccd1)
   
   | CPU profile % samples, Baseline  | CPU profile % samples, Candidate | 
   | :---                              |  :---          |   
    |0.90% 8259 `IndexingChain$PerField#invert`     | 1.00%  9162  `IndexingChain#getOrAddPerField`| 
    | 0.65% 5956 `IndexingChain#getOrAddField`    | 0.89%   8091  `IndexingChain#processDocument`    | 
    | 0.56% 5161 `IndexingChain#processField` | 0.69%  6255  `IndexingChain$PerField#invert` | 
    |   | 0.55%  5044  `IndexingChain$FieldSchema#<init>` | 
    |   | 0.52% 4744 `IndexingChain$FieldSchema#assertSameSchema` | 
   
   cc @jpountz 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r596373649



##########
File path: lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java
##########
@@ -149,9 +149,6 @@ boolean start(IndexableField field, boolean first) {
       doVectors = field.fieldType().storeTermVectors();
 
       if (doVectors) {
-
-        termsWriter.hasVectors = true;

Review comment:
       We do this now on the FieldInfo creation, in the `IndexingChain.PerField.setInvertState`: 
   ```java
   if (fieldInfo.hasVectors()) {
     termVectorsWriter.setHasVectors();
   }
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on pull request #11:
URL: https://github.com/apache/lucene/pull/11#issuecomment-803596283


   @s1monw Thank you for a thorough review! I've tried to address your comments. Can you please continue the review when you have available time.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r598290999



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexingChain.java
##########
@@ -1313,4 +1307,118 @@ public void recycleIntBlocks(int[][] blocks, int offset, int length) {
       bytesUsed.addAndGet(-(length * (IntBlockPool.INT_BLOCK_SIZE * Integer.BYTES)));
     }
   }
+
+  /**
+   * A schema of the field in the current document. With every new document this schema is reset. As
+   * the document fields are processed, we update the schema with options encountered in this
+   * document. Once the processing for the document is done, we compare the built schema of the
+   * current document with the corresponding FieldInfo (FieldInfo is built on a first document in
+   * the segment where we encounter this field). If there is inconsistency, we raise an error. This
+   * ensures that a field has the same data structures across all documents.
+   */
+  private static final class FieldSchema {

Review comment:
       @s1monw  Yes, our intention of `FieldInfo` to be set up by the 1st doc in the index that contains this field, and after that never to be changed for the whole index. Only `attributes` can be combined between several segments.
   It would be great  in future to make `FieldInfo` fully immutable .
   
   But `FieldSchema` has a different purpose. It is reset with every document and is being built as we encounter IndexableFields in a doc.  After that we compare the built `FieldSchema` with the expected `FieldInfo` for extra or missing fields.  For example, if `FieldInfo` for `field1` is set up  to be indexed with docValues and points,  but the `docX` has built `FieldSchema`  for `field1` that contain only docValues , we raise error and abort the indexing of docX. 
   
   Please let me know if this makes sense or you see how it can be organized better.
   
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r607913534



##########
File path: lucene/core/src/test/org/apache/lucene/document/TestPerFieldConsistency.java
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.document;
+
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomDouble;
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomFloat;
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomInt;
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomIntBetween;
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomLong;
+
+import com.carrotsearch.randomizedtesting.generators.RandomPicks;
+import java.io.IOException;
+import java.util.Random;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestPerFieldConsistency extends LuceneTestCase {
+
+  private static Field randomIndexedField(Random random, String fieldName) {
+    FieldType fieldType = new FieldType();
+    IndexOptions indexOptions = RandomPicks.randomFrom(random, IndexOptions.values());
+    while (indexOptions == IndexOptions.NONE) {
+      indexOptions = RandomPicks.randomFrom(random, IndexOptions.values());
+    }
+    fieldType.setIndexOptions(indexOptions);
+    fieldType.setStoreTermVectors(random.nextBoolean());
+    if (fieldType.storeTermVectors()) {
+      fieldType.setStoreTermVectorPositions(random.nextBoolean());
+      if (fieldType.storeTermVectorPositions()) {
+        fieldType.setStoreTermVectorPayloads(random.nextBoolean());
+        fieldType.setStoreTermVectorOffsets(random.nextBoolean());
+      }
+    }
+    fieldType.setOmitNorms(random.nextBoolean());
+    fieldType.setStored(random.nextBoolean());
+    fieldType.freeze();
+
+    return new Field(fieldName, "randomValue", fieldType);
+  }
+
+  private static Field randomPointField(Random random, String fieldName) {
+    switch (random.nextInt(4)) {
+      case 0:
+        return new LongPoint(fieldName, randomLong());
+      case 1:
+        return new IntPoint(fieldName, randomInt());
+      case 2:
+        return new DoublePoint(fieldName, randomDouble());
+      default:
+        return new FloatPoint(fieldName, randomFloat());
+    }
+  }
+
+  private static Field randomDocValuesField(Random random, String fieldName) {
+    switch (random.nextInt(4)) {
+      case 0:
+        return new BinaryDocValuesField(fieldName, new BytesRef("randomValue"));
+      case 1:
+        return new NumericDocValuesField(fieldName, randomLong());
+      case 2:
+        return new DoubleDocValuesField(fieldName, randomDouble());
+      default:
+        return new SortedSetDocValuesField(fieldName, new BytesRef("randomValue"));
+    }
+  }
+
+  private static Field randomVectorField(Random random, String fieldName) {
+    VectorValues.SearchStrategy searchStrategy =
+        RandomPicks.randomFrom(random, VectorValues.SearchStrategy.values());
+    while (searchStrategy == VectorValues.SearchStrategy.NONE) {
+      searchStrategy = RandomPicks.randomFrom(random, VectorValues.SearchStrategy.values());
+    }
+    float[] values = new float[randomIntBetween(1, 10)];
+    for (int i = 0; i < values.length; i++) {
+      values[i] = randomFloat();
+    }
+    return new VectorField(fieldName, values, searchStrategy);
+  }
+
+  private static Field[] randomFieldsWithTheSameName(String fieldName) {
+    final Field textField = randomIndexedField(random(), fieldName);
+    final Field docValuesField = randomDocValuesField(random(), fieldName);
+    final Field pointField = randomPointField(random(), fieldName);
+    final Field vectorField = randomVectorField(random(), fieldName);
+    return new Field[] {textField, docValuesField, pointField, vectorField};
+  }
+
+  public void testDocWithMissingSchemaOptionsThrowsError() throws IOException {
+    try (Directory dir = newDirectory();
+        IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig()); ) {
+      final Field[] fields = randomFieldsWithTheSameName("myfield");
+      final Document doc0 = new Document();
+      for (Field field : fields) {
+        doc0.add(field);
+      }
+      writer.addDocument(doc0);
+
+      // the same segment: indexing a doc with a missing field throws error
+      int missingFieldIdx = randomIntBetween(0, fields.length - 1);
+      final Document doc1 = new Document();
+      for (int i = 0; i < fields.length; i++) {
+        if (i != missingFieldIdx) {
+          doc1.add(fields[i]);
+        }
+      }
+      IllegalArgumentException exception =
+          expectThrows(IllegalArgumentException.class, () -> writer.addDocument(doc1));
+      String expectedErrMsg =
+          "Inconsistency of field data structures across documents for field [myfield] of doc [1].";
+      assertEquals(expectedErrMsg, exception.getMessage());
+
+      writer.flush();
+      try (IndexReader reader = DirectoryReader.open(writer)) {
+        LeafReader lr1 = reader.leaves().get(0).reader();
+        assertEquals(1, lr1.numDocs());
+        assertEquals(1, lr1.numDeletedDocs());
+      }
+
+      // diff segment, same index: indexing a doc with a missing field throws error
+      exception = expectThrows(IllegalArgumentException.class, () -> writer.addDocument(doc1));
+      assertTrue(exception.getMessage().contains("cannot change field \"myfield\" from "));
+
+      writer.addDocument(doc0); // add document with correct data structures
+
+      writer.flush();
+      try (IndexReader reader = DirectoryReader.open(writer)) {
+        LeafReader lr2 = reader.leaves().get(1).reader();
+        assertEquals(1, lr2.numDocs());
+        assertEquals(1, lr2.numDeletedDocs());
+      }
+    }
+  }
+
+  public void testDocWithExtraIndexingOptionsThrowsError() throws IOException {
+    try (Directory dir = newDirectory();
+        IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig())) {
+      final Field[] fields = randomFieldsWithTheSameName("myfield");
+      final Document doc0 = new Document();
+      int existingFieldIdx = randomIntBetween(0, fields.length - 1);
+      doc0.add(fields[existingFieldIdx]);
+      writer.addDocument(doc0);
+
+      // the same segment: indexing a field with extra field indexing options returns error
+      int extraFieldIndex = randomIntBetween(0, fields.length - 1);
+      while (extraFieldIndex == existingFieldIdx) {
+        extraFieldIndex = randomIntBetween(0, fields.length - 1);
+      }

Review comment:
       @jpountz Thanks, Adrien.  Addressed in in 8fe59c1




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r598287461



##########
File path: lucene/MIGRATE.md
##########
@@ -358,8 +358,10 @@ Rather, an IllegalArgumentException shall be thrown. This is introduced for bett
 defence and to ensure that there is no bubbling up of errors when Lucene is
 used in multi level applications
 
-## Assumption of data consistency between different data-structures sharing the same field name
+### Require consistency between data-structures on a per-field basis
 
-Sorting on a numeric field that is indexed with both doc values and points may use an
-optimization to skip non-competitive documents. This optimization relies on the assumption
-that the same data is stored in these points and doc values.
+A field must be indexed with the same index options and data-structures across 
+all documents within an index. Thus, for example, it is not allowed to have 
+one document in a index where a certain field is indexed with doc values 
+and points, and another document where the same field is indexed only with 
+points.

Review comment:
       Addressed in 6dc9f27 and 55fa8c8




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on pull request #11:
URL: https://github.com/apache/lucene/pull/11#issuecomment-805296177


   I've run indexing benchmarking using [luceneutil](https://github.com/mikemccand/luceneutil). And here are the results:
   
   - indexing time in ms
   - baseline: master branch
   - candidate: this PR
   
   
   | Dataset        | Baseline  | Candidate | Difference |
   | :---           |     ---:  |      ---: |      ---:  |
   | wikimedium500k | 98387     | 106189    | 7.9%       |
   | wikimedium1m   | 163054    | 167526    | 2.7%       |
   | wikimedium10m  | 1288031   | 1305160   | 1.3%       |
   
   cc @jpountz 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova edited a comment on pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova edited a comment on pull request #11:
URL: https://github.com/apache/lucene/pull/11#issuecomment-805296177


   I've run indexing benchmarking using [luceneutil](https://github.com/mikemccand/luceneutil). And here are the results:
   
   - indexing time in ms
   - baseline: master branch
   - candidate: this PR
   
   
   | Dataset        | Baseline  | Candidate | Difference |
   | :---           |     ---:  |      ---: |      ---:  |
   | wikimedium500k | 98387     | 106189    | 7.9%       |
   | wikimedium1m   | 174246    | 177075    | 1.6%       |
   | wikimedium10m  | 1356184   | 1359149   | 0.2%       |
   
   
   [wikimedium1m profiler](https://gist.github.com/mayya-sharipova/b4c8f47165a4bde8d2625487d2132319)
   [wikimedium10m profiler](https://gist.github.com/mayya-sharipova/68cf6d543863029777ad3028c662ccd1)
   
   cc @jpountz 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r596378537



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -130,127 +167,252 @@ public boolean checkConsistency() {
       }
     }
 
-    if (pointDimensionCount < 0) {
+    if (docValuesType == null) {
+      throw new IllegalStateException("DocValuesType must not be null (field: '" + name + "')");
+    }
+    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
       throw new IllegalStateException(
-          "pointDimensionCount must be >= 0; got " + pointDimensionCount);
+          "field '"
+              + name
+              + "' cannot have a docvalues update generation without having docvalues");
     }
 
+    if (pointDimensionCount < 0) {
+      throw new IllegalStateException(
+          "pointDimensionCount must be >= 0; got "
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
+    }
     if (pointIndexDimensionCount < 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be >= 0; got " + pointIndexDimensionCount);

Review comment:
       Unfortunately I can't do much with formatting. This is how ./gradlew :lucene:core:spotlessApply reformats the code.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] rmuir commented on pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
rmuir commented on pull request #11:
URL: https://github.com/apache/lucene/pull/11#issuecomment-801583847


   this problem is a hard one, it makes the manual review difficult. 
   
   I think there is a pre-existing condition around this stuff with the tests, they are weak. It is part of what makes the review take too long, we can't trust the tests, so we try to cope in another way. No, its not fair to ask this problem to be fixed as part of this PR, I just want us to keep it in mind.
   
   there 2 new tests added here, but they a bit difficult to grok, for example each test has a text field, docvalues field, points field, vectors field.
   
   Could we arrange the tests better to make it easy to write lots of simpler tests?
   
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r596299038



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -796,167 +565,97 @@ public void add(FieldInfos other) {
     /** Create a new field, or return existing one. */
     public FieldInfo getOrAdd(String name) {
       FieldInfo fi = fieldInfo(name);
-      if (fi == null) {
-        assert assertNotFinished();
-        // This field wasn't yet added to this in-RAM
-        // segment's FieldInfo, so now we get a global
-        // number for this field.  If the field was seen
-        // before then we'll get the same name and number,
-        // else we'll allocate a new one:
-        final boolean isSoftDeletesField = name.equals(globalFieldNumbers.softDeletesFieldName);
-        final int fieldNumber =
-            globalFieldNumbers.addOrGet(
-                name,
-                -1,
-                IndexOptions.NONE,
-                DocValuesType.NONE,
-                0,
-                0,
-                0,
-                0,
-                VectorValues.SearchStrategy.NONE,
-                isSoftDeletesField);
-        fi =
-            new FieldInfo(
-                name,
-                fieldNumber,
-                false,
-                false,
-                false,
-                IndexOptions.NONE,
-                DocValuesType.NONE,
-                -1,
-                new HashMap<>(),
-                0,
-                0,
-                0,
-                0,
-                VectorValues.SearchStrategy.NONE,
-                isSoftDeletesField);
-        assert !byName.containsKey(fi.name);
-        globalFieldNumbers.verifyConsistent(
-            Integer.valueOf(fi.number), fi.name, DocValuesType.NONE);
-        byName.put(fi.name, fi);
+      if (fi != null) {
+        return fi;
+      } else {
+        return addField(
+            name,
+            -1,
+            false,
+            false,
+            false,
+            IndexOptions.NONE,
+            DocValuesType.NONE,
+            -1,
+            new HashMap<>(),
+            0,
+            0,
+            0,
+            0,
+            VectorValues.SearchStrategy.NONE,
+            name.equals(globalFieldNumbers.softDeletesFieldName));
       }
+    }
 
-      return fi;
+    public FieldInfo add(FieldInfo fi) {
+      return add(fi, -1);
+    }
+
+    public FieldInfo add(FieldInfo fi, long dvGen) {
+      // IMPORTANT - reuse the field number if possible for consistent field numbers across segments
+      if (fi.getDocValuesType() == null) {
+        throw new NullPointerException("DocValuesType must not be null");
+      }
+      final FieldInfo curFi = fieldInfo(fi.name);
+      if (curFi == null) {
+        // original attributes is UnmodifiableMap
+        Map<String, String> attributes =
+            fi.attributes() == null ? null : new HashMap<>(fi.attributes());
+        return addField(
+            fi.name,
+            fi.number,
+            fi.hasVectors(),
+            fi.omitsNorms(),
+            fi.hasPayloads(),
+            fi.getIndexOptions(),
+            fi.getDocValuesType(),
+            dvGen,
+            attributes,
+            fi.getPointDimensionCount(),
+            fi.getPointIndexDimensionCount(),
+            fi.getPointNumBytes(),
+            fi.getVectorDimension(),
+            fi.getVectorSearchStrategy(),
+            fi.isSoftDeletesField());
+      } else {
+        curFi.verifySameSchema(fi, dvGen);
+        if (fi.attributes() != null) {
+          fi.attributes().forEach((k, v) -> curFi.putAttribute(k, v));
+        }
+        return curFi;
+      }
     }
 
-    private FieldInfo addOrUpdateInternal(
+    public FieldInfo add(

Review comment:
       Unfortunately I can't do anything with formatting. This is how `./gradlew :lucene:core:spotlessApply` reformats the code. 
   If I do the modifications as you suggested, `:lucene:core:spotlessJavaCheck` which is a part of `precommit` will not pass. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jtibshirani commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
jtibshirani commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r607396127



##########
File path: lucene/MIGRATE.md
##########
@@ -358,11 +358,21 @@ Rather, an IllegalArgumentException shall be thrown. This is introduced for bett
 defence and to ensure that there is no bubbling up of errors when Lucene is
 used in multi level applications
 
-## Assumption of data consistency between different data-structures sharing the same field name
+### Require consistency between data-structures on a per-field basis
 
-Sorting on a numeric field that is indexed with both doc values and points may use an

Review comment:
       I don't think this change enforces that the same values were passed to each field. So maybe we could keep this note about data consistency, as it's not fully covered under the new one "Require consistency between data structures..." I'm also curious if we plan to enforce value consistency in a follow-up? (Sorry if I'm missing something, I'm not fully up-to-date on the PR).




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova edited a comment on pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova edited a comment on pull request #11:
URL: https://github.com/apache/lucene/pull/11#issuecomment-805296177


   I've run indexing benchmarking using [luceneutil](https://github.com/mikemccand/luceneutil). And here are the results:
   
   - indexing time in ms
   - baseline: master branch
   - candidate: this PR
   
   
   | Dataset        | Baseline  | Candidate | Difference |
   | :---           |     ---:  |      ---: |      ---:  |
   | wikimedium500k | 98387     | 106189    | 7.9%       |
   | wikimedium1m   | 174246    | 177075    | 1.6%       |
   | wikimedium10m  | 1356184   | 1359149   | 0.2%       |
   
   ---
   
   [wikimedium1m profiler](https://gist.github.com/mayya-sharipova/b4c8f47165a4bde8d2625487d2132319)
   
   ---
   [wikimedium10m profiler](https://gist.github.com/mayya-sharipova/68cf6d543863029777ad3028c662ccd1):
   
   Extracting from CPU profiler everything related to `IndexingChain`, we can see that in **Candidate** there is an overhead spent on `assertSameSchema` that is a part of `processDocument`.
   
   | CPU profile % samples, Baseline  | CPU profile % samples, Candidate | 
   | :---                              |  :---          |   
    |0.90% 8259 `IndexingChain$PerField#invert`     | 1.00%  9162  `IndexingChain#getOrAddPerField`| 
    | 0.65% 5956 `IndexingChain#getOrAddField`    | 0.89%   8091  `IndexingChain#processDocument`    | 
    | 0.56% 5161 `IndexingChain#processField` | 0.69%  6255  `IndexingChain$PerField#invert` | 
    |   | 0.55%  5044  `IndexingChain$FieldSchema#<init>` | 
    |   | 0.52% 4744 `IndexingChain$FieldSchema#assertSameSchema` | 
   
   cc @jpountz 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r607913700



##########
File path: lucene/core/src/test/org/apache/lucene/document/TestPerFieldConsistency.java
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.document;
+
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomDouble;
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomFloat;
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomInt;
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomIntBetween;
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomLong;
+
+import com.carrotsearch.randomizedtesting.generators.RandomPicks;
+import java.io.IOException;
+import java.util.Random;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestPerFieldConsistency extends LuceneTestCase {
+
+  private static Field randomIndexedField(Random random, String fieldName) {
+    FieldType fieldType = new FieldType();
+    IndexOptions indexOptions = RandomPicks.randomFrom(random, IndexOptions.values());
+    while (indexOptions == IndexOptions.NONE) {
+      indexOptions = RandomPicks.randomFrom(random, IndexOptions.values());
+    }
+    fieldType.setIndexOptions(indexOptions);
+    fieldType.setStoreTermVectors(random.nextBoolean());
+    if (fieldType.storeTermVectors()) {
+      fieldType.setStoreTermVectorPositions(random.nextBoolean());
+      if (fieldType.storeTermVectorPositions()) {
+        fieldType.setStoreTermVectorPayloads(random.nextBoolean());
+        fieldType.setStoreTermVectorOffsets(random.nextBoolean());
+      }
+    }
+    fieldType.setOmitNorms(random.nextBoolean());
+    fieldType.setStored(random.nextBoolean());
+    fieldType.freeze();
+
+    return new Field(fieldName, "randomValue", fieldType);
+  }
+
+  private static Field randomPointField(Random random, String fieldName) {
+    switch (random.nextInt(4)) {
+      case 0:
+        return new LongPoint(fieldName, randomLong());
+      case 1:
+        return new IntPoint(fieldName, randomInt());
+      case 2:
+        return new DoublePoint(fieldName, randomDouble());
+      default:
+        return new FloatPoint(fieldName, randomFloat());
+    }
+  }
+
+  private static Field randomDocValuesField(Random random, String fieldName) {
+    switch (random.nextInt(4)) {
+      case 0:
+        return new BinaryDocValuesField(fieldName, new BytesRef("randomValue"));
+      case 1:
+        return new NumericDocValuesField(fieldName, randomLong());
+      case 2:
+        return new DoubleDocValuesField(fieldName, randomDouble());
+      default:
+        return new SortedSetDocValuesField(fieldName, new BytesRef("randomValue"));
+    }
+  }
+
+  private static Field randomVectorField(Random random, String fieldName) {
+    VectorValues.SearchStrategy searchStrategy =
+        RandomPicks.randomFrom(random, VectorValues.SearchStrategy.values());
+    while (searchStrategy == VectorValues.SearchStrategy.NONE) {
+      searchStrategy = RandomPicks.randomFrom(random, VectorValues.SearchStrategy.values());
+    }
+    float[] values = new float[randomIntBetween(1, 10)];
+    for (int i = 0; i < values.length; i++) {
+      values[i] = randomFloat();
+    }
+    return new VectorField(fieldName, values, searchStrategy);
+  }
+
+  private static Field[] randomFieldsWithTheSameName(String fieldName) {
+    final Field textField = randomIndexedField(random(), fieldName);
+    final Field docValuesField = randomDocValuesField(random(), fieldName);
+    final Field pointField = randomPointField(random(), fieldName);
+    final Field vectorField = randomVectorField(random(), fieldName);
+    return new Field[] {textField, docValuesField, pointField, vectorField};
+  }
+
+  public void testDocWithMissingSchemaOptionsThrowsError() throws IOException {
+    try (Directory dir = newDirectory();
+        IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig()); ) {
+      final Field[] fields = randomFieldsWithTheSameName("myfield");
+      final Document doc0 = new Document();
+      for (Field field : fields) {
+        doc0.add(field);
+      }
+      writer.addDocument(doc0);
+
+      // the same segment: indexing a doc with a missing field throws error
+      int missingFieldIdx = randomIntBetween(0, fields.length - 1);

Review comment:
       Addressed in in 8fe59c1




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r596387798



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -130,127 +167,252 @@ public boolean checkConsistency() {
       }
     }
 
-    if (pointDimensionCount < 0) {
+    if (docValuesType == null) {
+      throw new IllegalStateException("DocValuesType must not be null (field: '" + name + "')");
+    }
+    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
       throw new IllegalStateException(
-          "pointDimensionCount must be >= 0; got " + pointDimensionCount);
+          "field '"
+              + name
+              + "' cannot have a docvalues update generation without having docvalues");
     }
 
+    if (pointDimensionCount < 0) {
+      throw new IllegalStateException(
+          "pointDimensionCount must be >= 0; got "
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
+    }
     if (pointIndexDimensionCount < 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be >= 0; got " + pointIndexDimensionCount);
+          "pointIndexDimensionCount must be >= 0; got "
+              + pointIndexDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes < 0) {
-      throw new IllegalStateException("pointNumBytes must be >= 0; got " + pointNumBytes);
+      throw new IllegalStateException(
+          "pointNumBytes must be >= 0; got " + pointNumBytes + " (field: '" + name + "')");
     }
 
     if (pointDimensionCount != 0 && pointNumBytes == 0) {
       throw new IllegalStateException(
-          "pointNumBytes must be > 0 when pointDimensionCount=" + pointDimensionCount);
+          "pointNumBytes must be > 0 when pointDimensionCount="
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointIndexDimensionCount != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be 0 when pointDimensionCount=0");
+          "pointIndexDimensionCount must be 0 when pointDimensionCount=0"
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointDimensionCount must be > 0 when pointNumBytes=" + pointNumBytes);
+          "pointDimensionCount must be > 0 when pointNumBytes="
+              + pointNumBytes
+              + " (field: '"
+              + name
+              + "')");
     }
 
-    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
+    if (vectorSearchStrategy == null) {
       throw new IllegalStateException(
-          "field '"
-              + name
-              + "' cannot have a docvalues update generation without having docvalues");
+          "Vector search strategy must not be null (field: '" + name + "')");
     }
-
     if (vectorDimension < 0) {
-      throw new IllegalStateException("vectorDimension must be >=0; got " + vectorDimension);
+      throw new IllegalStateException(
+          "vectorDimension must be >=0; got " + vectorDimension + " (field: '" + name + "')");
     }
-
     if (vectorDimension == 0 && vectorSearchStrategy != VectorValues.SearchStrategy.NONE) {
       throw new IllegalStateException(
-          "vector search strategy must be NONE when dimension = 0; got " + vectorSearchStrategy);
+          "vector search strategy must be NONE when dimension = 0; got "
+              + vectorSearchStrategy
+              + " (field: '"
+              + name
+              + "')");
     }
-
     return true;
   }
 
-  // should only be called by FieldInfos#addOrUpdate
-  void update(
-      boolean storeTermVector,
-      boolean omitNorms,
-      boolean storePayloads,
-      IndexOptions indexOptions,
-      Map<String, String> attributes,
-      int dimensionCount,
-      int indexDimensionCount,
-      int dimensionNumBytes) {
-    if (indexOptions == null) {
-      throw new NullPointerException("IndexOptions must not be null (field: \"" + name + "\")");
-    }
-    // System.out.println("FI.update field=" + name + " indexed=" + indexed + " omitNorms=" +
-    // omitNorms + " this.omitNorms=" + this.omitNorms);
-    if (this.indexOptions != indexOptions) {
-      if (this.indexOptions == IndexOptions.NONE) {
-        this.indexOptions = indexOptions;
-      } else if (indexOptions != IndexOptions.NONE) {
-        throw new IllegalArgumentException(
-            "cannot change field \""
-                + name
-                + "\" from index options="
-                + this.indexOptions
-                + " to inconsistent index options="
-                + indexOptions);
-      }
+  void verifySameSchema(FieldInfo o, long dvGen) {
+    String fieldName = this.name;
+    verifySameIndexOptions(fieldName, this.indexOptions, o.getIndexOptions());
+    if (this.indexOptions != IndexOptions.NONE) {
+      verifySameOmitNorms(fieldName, this.omitNorms, o.omitNorms);
+      verifySameStoreTermVectors(fieldName, this.storeTermVector, o.storeTermVector);
+    }
+    verifySameDocValuesType(fieldName, this.docValuesType, o.docValuesType);
+    verifySameDVGen(fieldName, this.dvGen, dvGen);
+    verifySamePointsOptions(
+        fieldName,
+        this.pointDimensionCount,
+        this.pointIndexDimensionCount,
+        this.pointNumBytes,
+        o.pointDimensionCount,
+        o.pointIndexDimensionCount,
+        o.pointNumBytes);
+    verifySameVectorOptions(
+        fieldName,
+        this.vectorDimension,
+        this.vectorSearchStrategy,
+        o.vectorDimension,
+        o.vectorSearchStrategy);
+  }
+
+  /**
+   * Verify that the provided index options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameIndexOptions(
+      String fieldName, IndexOptions indexOptions1, IndexOptions indexOptions2) {
+    if (indexOptions1 != indexOptions2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from index options="
+              + indexOptions1
+              + " to inconsistent index options="
+              + indexOptions2);
     }
+  }
 
-    if (this.pointDimensionCount == 0 && dimensionCount != 0) {
-      this.pointDimensionCount = dimensionCount;
-      this.pointIndexDimensionCount = indexDimensionCount;
-      this.pointNumBytes = dimensionNumBytes;
-    } else if (dimensionCount != 0
-        && (this.pointDimensionCount != dimensionCount
-            || this.pointIndexDimensionCount != indexDimensionCount
-            || this.pointNumBytes != dimensionNumBytes)) {
+  /**
+   * Verify that the provided docValues type are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameDocValuesType(
+      String fieldName, DocValuesType docValuesType1, DocValuesType docValuesType2) {
+    if (docValuesType1 != docValuesType2) {
       throw new IllegalArgumentException(
           "cannot change field \""
-              + name
+              + fieldName
+              + "\" from doc values type="
+              + docValuesType1
+              + " to inconsistent doc values type="
+              + docValuesType2);
+    }
+  }
+
+  /**
+   * Verify that the provided doc values generations are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  // TODO: not sure if gen also must be the same
+  public static void verifySameDVGen(String fieldName, long docValuesGen1, long docValuesGen2) {
+    if (docValuesGen1 != docValuesGen2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from doc values generation="
+              + docValuesGen1
+              + " to inconsistent doc values generation="
+              + docValuesGen2);
+    }
+  }
+
+  /**
+   * Verify that the provided store term vectors options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameStoreTermVectors(
+      String fieldName, boolean storeTermVector1, boolean storeTermVector2) {
+    if (storeTermVector1 != storeTermVector2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from storeTermVector="
+              + storeTermVector1
+              + " to inconsistent storeTermVector="
+              + storeTermVector2);
+    }
+  }
+
+  /**
+   * Verify that the provided omitNorms are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameOmitNorms(String fieldName, boolean omitNorms1, boolean omitNorms2) {
+    if (omitNorms1 != omitNorms2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from omitNorms="
+              + omitNorms1
+              + " to inconsistent omitNorms="
+              + omitNorms2);
+    }
+  }
+
+  /**
+   * Verify that the provided points indexing options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySamePointsOptions(
+      String fieldName,
+      int pointDimensionCount1,
+      int indexDimensionCount1,
+      int numBytes1,
+      int pointDimensionCount2,
+      int indexDimensionCount2,
+      int numBytes2) {
+    if (pointDimensionCount1 != pointDimensionCount2
+        || indexDimensionCount1 != indexDimensionCount2
+        || numBytes1 != numBytes2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
               + "\" from points dimensionCount="
-              + this.pointDimensionCount
+              + pointDimensionCount1
               + ", indexDimensionCount="
-              + this.pointIndexDimensionCount
+              + indexDimensionCount1
               + ", numBytes="
-              + this.pointNumBytes
+              + numBytes1
               + " to inconsistent dimensionCount="
-              + dimensionCount
+              + pointDimensionCount2
               + ", indexDimensionCount="
-              + indexDimensionCount
+              + indexDimensionCount2
               + ", numBytes="
-              + dimensionNumBytes);
+              + numBytes2);
     }
+  }
 
-    // if updated field data is not for indexing, leave the updates out
-    if (this.indexOptions != IndexOptions.NONE) {
-      this.storeTermVector |= storeTermVector; // once vector, always vector
-      this.storePayloads |= storePayloads;
-
-      // Awkward: only drop norms if incoming update is indexed:
-      if (indexOptions != IndexOptions.NONE && this.omitNorms != omitNorms) {
-        this.omitNorms = true; // if one require omitNorms at least once, it remains off for life
-      }
-    }
-    if (this.indexOptions == IndexOptions.NONE
-        || this.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
-      // cannot store payloads if we don't store positions:
-      this.storePayloads = false;
-    }
-    if (attributes != null) {
-      this.attributes.putAll(attributes);
+  /**
+   * Verify that the provided vector indexing options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameVectorOptions(

Review comment:
       Addressed in 0fe3493110ac2a5f750ad41f732436daff6c69f5




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r598288749



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -972,6 +671,64 @@ private boolean assertNotFinished() {
       return true;
     }
 
+    private FieldInfo addField(
+        String name,
+        int preferredFieldNumber,
+        boolean storeTermVector,
+        boolean omitNorms,
+        boolean storePayloads,
+        IndexOptions indexOptions,
+        DocValuesType docValues,
+        long dvGen,
+        Map<String, String> attributes,
+        int dataDimensionCount,
+        int indexDimensionCount,
+        int dimensionNumBytes,
+        int vectorDimension,
+        VectorValues.SearchStrategy vectorSearchStrategy,
+        boolean isSoftDeletesField) {
+      // This field wasn't yet added to this in-RAM
+      // segment's FieldInfo, so now we get a global
+      // number for this field.  If the field was seen
+      // before then we'll get the same name and number,
+      // else we'll allocate a new one:
+      assert assertNotFinished();
+      final int fieldNumber =
+          globalFieldNumbers.addOrGet(

Review comment:
       Addressed in 6dc9f27 and 55fa8c8

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -339,110 +350,17 @@ synchronized int addOrGet(
         String fieldName,
         int preferredFieldNumber,
         IndexOptions indexOptions,
+        boolean storeTermVector,

Review comment:
       Addressed in 6dc9f27 and 55fa8c8




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r596388235



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -130,127 +167,252 @@ public boolean checkConsistency() {
       }
     }
 
-    if (pointDimensionCount < 0) {
+    if (docValuesType == null) {
+      throw new IllegalStateException("DocValuesType must not be null (field: '" + name + "')");
+    }
+    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
       throw new IllegalStateException(
-          "pointDimensionCount must be >= 0; got " + pointDimensionCount);
+          "field '"
+              + name
+              + "' cannot have a docvalues update generation without having docvalues");
     }
 
+    if (pointDimensionCount < 0) {
+      throw new IllegalStateException(
+          "pointDimensionCount must be >= 0; got "
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
+    }
     if (pointIndexDimensionCount < 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be >= 0; got " + pointIndexDimensionCount);
+          "pointIndexDimensionCount must be >= 0; got "
+              + pointIndexDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes < 0) {
-      throw new IllegalStateException("pointNumBytes must be >= 0; got " + pointNumBytes);
+      throw new IllegalStateException(
+          "pointNumBytes must be >= 0; got " + pointNumBytes + " (field: '" + name + "')");
     }
 
     if (pointDimensionCount != 0 && pointNumBytes == 0) {
       throw new IllegalStateException(
-          "pointNumBytes must be > 0 when pointDimensionCount=" + pointDimensionCount);
+          "pointNumBytes must be > 0 when pointDimensionCount="
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointIndexDimensionCount != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be 0 when pointDimensionCount=0");
+          "pointIndexDimensionCount must be 0 when pointDimensionCount=0"
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointDimensionCount must be > 0 when pointNumBytes=" + pointNumBytes);
+          "pointDimensionCount must be > 0 when pointNumBytes="
+              + pointNumBytes
+              + " (field: '"
+              + name
+              + "')");
     }
 
-    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
+    if (vectorSearchStrategy == null) {
       throw new IllegalStateException(
-          "field '"
-              + name
-              + "' cannot have a docvalues update generation without having docvalues");
+          "Vector search strategy must not be null (field: '" + name + "')");
     }
-
     if (vectorDimension < 0) {
-      throw new IllegalStateException("vectorDimension must be >=0; got " + vectorDimension);
+      throw new IllegalStateException(
+          "vectorDimension must be >=0; got " + vectorDimension + " (field: '" + name + "')");
     }
-
     if (vectorDimension == 0 && vectorSearchStrategy != VectorValues.SearchStrategy.NONE) {
       throw new IllegalStateException(
-          "vector search strategy must be NONE when dimension = 0; got " + vectorSearchStrategy);
+          "vector search strategy must be NONE when dimension = 0; got "
+              + vectorSearchStrategy
+              + " (field: '"
+              + name
+              + "')");
     }
-
     return true;
   }
 
-  // should only be called by FieldInfos#addOrUpdate
-  void update(
-      boolean storeTermVector,
-      boolean omitNorms,
-      boolean storePayloads,
-      IndexOptions indexOptions,
-      Map<String, String> attributes,
-      int dimensionCount,
-      int indexDimensionCount,
-      int dimensionNumBytes) {
-    if (indexOptions == null) {
-      throw new NullPointerException("IndexOptions must not be null (field: \"" + name + "\")");
-    }
-    // System.out.println("FI.update field=" + name + " indexed=" + indexed + " omitNorms=" +
-    // omitNorms + " this.omitNorms=" + this.omitNorms);
-    if (this.indexOptions != indexOptions) {
-      if (this.indexOptions == IndexOptions.NONE) {
-        this.indexOptions = indexOptions;
-      } else if (indexOptions != IndexOptions.NONE) {
-        throw new IllegalArgumentException(
-            "cannot change field \""
-                + name
-                + "\" from index options="
-                + this.indexOptions
-                + " to inconsistent index options="
-                + indexOptions);
-      }
+  void verifySameSchema(FieldInfo o, long dvGen) {
+    String fieldName = this.name;
+    verifySameIndexOptions(fieldName, this.indexOptions, o.getIndexOptions());
+    if (this.indexOptions != IndexOptions.NONE) {
+      verifySameOmitNorms(fieldName, this.omitNorms, o.omitNorms);
+      verifySameStoreTermVectors(fieldName, this.storeTermVector, o.storeTermVector);
+    }
+    verifySameDocValuesType(fieldName, this.docValuesType, o.docValuesType);
+    verifySameDVGen(fieldName, this.dvGen, dvGen);
+    verifySamePointsOptions(
+        fieldName,
+        this.pointDimensionCount,
+        this.pointIndexDimensionCount,
+        this.pointNumBytes,
+        o.pointDimensionCount,
+        o.pointIndexDimensionCount,
+        o.pointNumBytes);
+    verifySameVectorOptions(
+        fieldName,
+        this.vectorDimension,
+        this.vectorSearchStrategy,
+        o.vectorDimension,
+        o.vectorSearchStrategy);
+  }
+
+  /**
+   * Verify that the provided index options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameIndexOptions(
+      String fieldName, IndexOptions indexOptions1, IndexOptions indexOptions2) {
+    if (indexOptions1 != indexOptions2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from index options="
+              + indexOptions1
+              + " to inconsistent index options="
+              + indexOptions2);
     }
+  }
 
-    if (this.pointDimensionCount == 0 && dimensionCount != 0) {
-      this.pointDimensionCount = dimensionCount;
-      this.pointIndexDimensionCount = indexDimensionCount;
-      this.pointNumBytes = dimensionNumBytes;
-    } else if (dimensionCount != 0
-        && (this.pointDimensionCount != dimensionCount
-            || this.pointIndexDimensionCount != indexDimensionCount
-            || this.pointNumBytes != dimensionNumBytes)) {
+  /**
+   * Verify that the provided docValues type are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameDocValuesType(
+      String fieldName, DocValuesType docValuesType1, DocValuesType docValuesType2) {
+    if (docValuesType1 != docValuesType2) {
       throw new IllegalArgumentException(
           "cannot change field \""
-              + name
+              + fieldName
+              + "\" from doc values type="
+              + docValuesType1
+              + " to inconsistent doc values type="
+              + docValuesType2);
+    }
+  }
+
+  /**
+   * Verify that the provided doc values generations are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  // TODO: not sure if gen also must be the same
+  public static void verifySameDVGen(String fieldName, long docValuesGen1, long docValuesGen2) {

Review comment:
       Addressed in 0fe3493110ac2a5f750ad41f732436daff6c69f5




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r596397825



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -339,110 +350,17 @@ synchronized int addOrGet(
         String fieldName,
         int preferredFieldNumber,
         IndexOptions indexOptions,
+        boolean storeTermVector,

Review comment:
       Thanks Simon, I will see how to redesign accordingly!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova edited a comment on pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova edited a comment on pull request #11:
URL: https://github.com/apache/lucene/pull/11#issuecomment-805296177


   I've run indexing benchmarking using [luceneutil](https://github.com/mikemccand/luceneutil). And here are the results:
   
   - indexing time in ms
   - baseline: master branch
   - candidate: this PR
   
   
   | Dataset        | Baseline  | Candidate | Difference |
   | :---           |     ---:  |      ---: |      ---:  |
   | wikimedium500k | 98387     | 106189    | 7.9%       |
   | wikimedium1m   | 174246    | 177075    | 1.6%       |
   | wikimedium10m  | 1356184   | 1359149   | 0.2%       |
   
   ---
   
   [wikimedium1m profiler](https://gist.github.com/mayya-sharipova/b4c8f47165a4bde8d2625487d2132319)
   
   | CPU profile % samples, Baseline  | CPU profile % samples, Candidate | 
   | :---                              |  :---          |   
    |0.73% 783 `IndexingChain$PerField#invert`     | 0.80%  864  `IndexingChain#getOrAddPerField`| 
    |     | 0.61% 658  `IndexingChain$FieldSchema#<init>`    | 
    |  | 0.58%  633  `IndexingChain#processField` | 
   
   
   ---
   [wikimedium10m profiler](https://gist.github.com/mayya-sharipova/68cf6d543863029777ad3028c662ccd1):
   
   Extracting from CPU profiler everything related to `IndexingChain`, we can see that in **Candidate** there is an overhead spent on `assertSameSchema` that is a part of `processDocument`.
   
   | CPU profile % samples, Baseline  | CPU profile % samples, Candidate | 
   | :---                              |  :---          |   
    |0.90% 8259 `IndexingChain$PerField#invert`     | 1.00%  9162  `IndexingChain#getOrAddPerField`| 
    | 0.65% 5956 `IndexingChain#getOrAddField`    | 0.89%   8091  `IndexingChain#processDocument`    | 
    | 0.56% 5161 `IndexingChain#processField` | 0.69%  6255  `IndexingChain$PerField#invert` | 
    |   | 0.55%  5044  `IndexingChain$FieldSchema#<init>` | 
    |   | 0.52% 4744 `IndexingChain$FieldSchema#assertSameSchema` | 
   
   cc @jpountz 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jpountz commented on pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
jpountz commented on pull request #11:
URL: https://github.com/apache/lucene/pull/11#issuecomment-813948862


   > I'm also curious if we plan to enforce value consistency in a follow-up?
   
   It would be great if we could enforce value consistency too. It feels more challenging because `IndexingChain`, where these checks are performed, only knows about the encoded representation of the data. For instance if you index a `LongPoint`, `IndexingChain` doesn't see a `long` but a `byte[8]`. So we'd need to hardcode expectations in `IndexingChain` about how the `byte[]` encoding of terms/points is expected to match the `long` encoding of numeric doc values for instance, which I don't feel great about. I believe that a better way of doing it would be to introduce a notion of schema in Lucene, so that you would be passing values to Lucene directly, and Lucene would take care of putting the right (encoded) data into the right data structures, but this would also be a much larger change.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r596388310



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -130,127 +167,252 @@ public boolean checkConsistency() {
       }
     }
 
-    if (pointDimensionCount < 0) {
+    if (docValuesType == null) {
+      throw new IllegalStateException("DocValuesType must not be null (field: '" + name + "')");
+    }
+    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
       throw new IllegalStateException(
-          "pointDimensionCount must be >= 0; got " + pointDimensionCount);
+          "field '"
+              + name
+              + "' cannot have a docvalues update generation without having docvalues");
     }
 
+    if (pointDimensionCount < 0) {
+      throw new IllegalStateException(
+          "pointDimensionCount must be >= 0; got "
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
+    }
     if (pointIndexDimensionCount < 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be >= 0; got " + pointIndexDimensionCount);
+          "pointIndexDimensionCount must be >= 0; got "
+              + pointIndexDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes < 0) {
-      throw new IllegalStateException("pointNumBytes must be >= 0; got " + pointNumBytes);
+      throw new IllegalStateException(
+          "pointNumBytes must be >= 0; got " + pointNumBytes + " (field: '" + name + "')");
     }
 
     if (pointDimensionCount != 0 && pointNumBytes == 0) {
       throw new IllegalStateException(
-          "pointNumBytes must be > 0 when pointDimensionCount=" + pointDimensionCount);
+          "pointNumBytes must be > 0 when pointDimensionCount="
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointIndexDimensionCount != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be 0 when pointDimensionCount=0");
+          "pointIndexDimensionCount must be 0 when pointDimensionCount=0"
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointDimensionCount must be > 0 when pointNumBytes=" + pointNumBytes);
+          "pointDimensionCount must be > 0 when pointNumBytes="
+              + pointNumBytes
+              + " (field: '"
+              + name
+              + "')");
     }
 
-    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
+    if (vectorSearchStrategy == null) {
       throw new IllegalStateException(
-          "field '"
-              + name
-              + "' cannot have a docvalues update generation without having docvalues");
+          "Vector search strategy must not be null (field: '" + name + "')");
     }
-
     if (vectorDimension < 0) {
-      throw new IllegalStateException("vectorDimension must be >=0; got " + vectorDimension);
+      throw new IllegalStateException(
+          "vectorDimension must be >=0; got " + vectorDimension + " (field: '" + name + "')");
     }
-
     if (vectorDimension == 0 && vectorSearchStrategy != VectorValues.SearchStrategy.NONE) {
       throw new IllegalStateException(
-          "vector search strategy must be NONE when dimension = 0; got " + vectorSearchStrategy);
+          "vector search strategy must be NONE when dimension = 0; got "
+              + vectorSearchStrategy
+              + " (field: '"
+              + name
+              + "')");
     }
-
     return true;
   }
 
-  // should only be called by FieldInfos#addOrUpdate
-  void update(
-      boolean storeTermVector,
-      boolean omitNorms,
-      boolean storePayloads,
-      IndexOptions indexOptions,
-      Map<String, String> attributes,
-      int dimensionCount,
-      int indexDimensionCount,
-      int dimensionNumBytes) {
-    if (indexOptions == null) {
-      throw new NullPointerException("IndexOptions must not be null (field: \"" + name + "\")");
-    }
-    // System.out.println("FI.update field=" + name + " indexed=" + indexed + " omitNorms=" +
-    // omitNorms + " this.omitNorms=" + this.omitNorms);
-    if (this.indexOptions != indexOptions) {
-      if (this.indexOptions == IndexOptions.NONE) {
-        this.indexOptions = indexOptions;
-      } else if (indexOptions != IndexOptions.NONE) {
-        throw new IllegalArgumentException(
-            "cannot change field \""
-                + name
-                + "\" from index options="
-                + this.indexOptions
-                + " to inconsistent index options="
-                + indexOptions);
-      }
+  void verifySameSchema(FieldInfo o, long dvGen) {
+    String fieldName = this.name;
+    verifySameIndexOptions(fieldName, this.indexOptions, o.getIndexOptions());
+    if (this.indexOptions != IndexOptions.NONE) {
+      verifySameOmitNorms(fieldName, this.omitNorms, o.omitNorms);
+      verifySameStoreTermVectors(fieldName, this.storeTermVector, o.storeTermVector);
+    }
+    verifySameDocValuesType(fieldName, this.docValuesType, o.docValuesType);
+    verifySameDVGen(fieldName, this.dvGen, dvGen);
+    verifySamePointsOptions(
+        fieldName,
+        this.pointDimensionCount,
+        this.pointIndexDimensionCount,
+        this.pointNumBytes,
+        o.pointDimensionCount,
+        o.pointIndexDimensionCount,
+        o.pointNumBytes);
+    verifySameVectorOptions(
+        fieldName,
+        this.vectorDimension,
+        this.vectorSearchStrategy,
+        o.vectorDimension,
+        o.vectorSearchStrategy);
+  }
+
+  /**
+   * Verify that the provided index options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameIndexOptions(
+      String fieldName, IndexOptions indexOptions1, IndexOptions indexOptions2) {
+    if (indexOptions1 != indexOptions2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from index options="
+              + indexOptions1
+              + " to inconsistent index options="
+              + indexOptions2);
     }
+  }
 
-    if (this.pointDimensionCount == 0 && dimensionCount != 0) {
-      this.pointDimensionCount = dimensionCount;
-      this.pointIndexDimensionCount = indexDimensionCount;
-      this.pointNumBytes = dimensionNumBytes;
-    } else if (dimensionCount != 0
-        && (this.pointDimensionCount != dimensionCount
-            || this.pointIndexDimensionCount != indexDimensionCount
-            || this.pointNumBytes != dimensionNumBytes)) {
+  /**
+   * Verify that the provided docValues type are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameDocValuesType(

Review comment:
       Addressed in 0fe3493110ac2a5f750ad41f732436daff6c69f5




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r598287420



##########
File path: lucene/CHANGES.txt
##########
@@ -92,6 +92,12 @@ API Changes
 * LUCENE-9480: Make DataInput's skipBytes(long) abstract as the implementation was not performant.
   IndexInput's api is unaffected: skipBytes() is implemented via seek(). (Greg Miller)
 
+* LUCENE-9334:  Require consistency between data-structures on a per-field basis.
+  A field across all documents within an index must be indexed with the same index
+  options and data-structures. As a consequence of this, doc values updates are
+  only applicable for fields that are indexed with doc values only

Review comment:
       Addressed in 6dc9f27 and 55fa8c8




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r598287872



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -972,6 +671,64 @@ private boolean assertNotFinished() {
       return true;
     }
 
+    private FieldInfo addField(

Review comment:
       I could not remove `add(FieldInfo fi, long dvGen)` as `TestParallelLeafReader::testWithDocValuesUpdates()` [expects dvGen to be unchanged](https://github.com/apache/lucene/blob/main/lucene/core/src/test/org/apache/lucene/index/TestParallelLeafReader.java#L408).




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova edited a comment on pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova edited a comment on pull request #11:
URL: https://github.com/apache/lucene/pull/11#issuecomment-805296177


   I've run indexing benchmarking using [luceneutil](https://github.com/mikemccand/luceneutil). And here are the results:
   
   - indexing time in ms
   - baseline: master branch
   - candidate: this PR
   
   
   | Dataset        | Baseline  | Candidate | Difference |
   | :---           |     ---:  |      ---: |      ---:  |
   | wikimedium500k | 98387     | 106189    | 7.9%       |
   | wikimedium1m   | 174246    | 177075    | 1.6%       |
   | wikimedium10m  | 1288031   | 1305160   | 1.3%       |
   
   
   [wikimedium1m profiler](https://gist.github.com/mayya-sharipova/b4c8f47165a4bde8d2625487d2132319)
   
   cc @jpountz 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r607913125



##########
File path: lucene/MIGRATE.md
##########
@@ -358,11 +358,21 @@ Rather, an IllegalArgumentException shall be thrown. This is introduced for bett
 defence and to ensure that there is no bubbling up of errors when Lucene is
 used in multi level applications
 
-## Assumption of data consistency between different data-structures sharing the same field name
+### Require consistency between data-structures on a per-field basis
 
-Sorting on a numeric field that is indexed with both doc values and points may use an

Review comment:
       @jtibshirani Thanks, that's a great point. I've restored a point about data consistency in 8fe59c1




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jpountz commented on pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
jpountz commented on pull request #11:
URL: https://github.com/apache/lucene/pull/11#issuecomment-811987053


   Thanks @mayya-sharipova for the benchmark. The overhead looks very reasonable to me, I don't think it should be a reason not to proceed with this change.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova edited a comment on pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova edited a comment on pull request #11:
URL: https://github.com/apache/lucene/pull/11#issuecomment-807089871






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r596374710



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -111,6 +111,43 @@ public FieldInfo(
 
   /** Performs internal consistency checks. Always returns true (or throws IllegalStateException) */
   public boolean checkConsistency() {
+    return checkOptionsCorrectness(
+        name,
+        storeTermVector,
+        omitNorms,
+        storePayloads,
+        indexOptions,
+        docValuesType,
+        dvGen,
+        pointDimensionCount,
+        pointIndexDimensionCount,
+        pointNumBytes,
+        vectorDimension,
+        vectorSearchStrategy);
+  }
+
+  /**
+   * Check correctness of FieldInfo options
+   *
+   * @throws IllegalStateException if some options are incorrect
+   * @return {@code true} if all options are correct
+   */
+  public static boolean checkOptionsCorrectness(
+      String name,

Review comment:
       Unfortunately I can't do anything with formatting. This is how `./gradlew :lucene:core:spotlessApply` reformats the code.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r598287549



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -130,127 +167,252 @@ public boolean checkConsistency() {
       }
     }
 
-    if (pointDimensionCount < 0) {
+    if (docValuesType == null) {
+      throw new IllegalStateException("DocValuesType must not be null (field: '" + name + "')");
+    }
+    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
       throw new IllegalStateException(
-          "pointDimensionCount must be >= 0; got " + pointDimensionCount);
+          "field '"
+              + name
+              + "' cannot have a docvalues update generation without having docvalues");
     }
 
+    if (pointDimensionCount < 0) {
+      throw new IllegalStateException(
+          "pointDimensionCount must be >= 0; got "

Review comment:
       Yes, it is pre-existing. But I still addressed this  in 6dc9f27 and 55fa8c8




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r598275683



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -796,167 +565,97 @@ public void add(FieldInfos other) {
     /** Create a new field, or return existing one. */
     public FieldInfo getOrAdd(String name) {
       FieldInfo fi = fieldInfo(name);
-      if (fi == null) {
-        assert assertNotFinished();
-        // This field wasn't yet added to this in-RAM
-        // segment's FieldInfo, so now we get a global
-        // number for this field.  If the field was seen
-        // before then we'll get the same name and number,
-        // else we'll allocate a new one:
-        final boolean isSoftDeletesField = name.equals(globalFieldNumbers.softDeletesFieldName);
-        final int fieldNumber =
-            globalFieldNumbers.addOrGet(
-                name,
-                -1,
-                IndexOptions.NONE,
-                DocValuesType.NONE,
-                0,
-                0,
-                0,
-                0,
-                VectorValues.SearchStrategy.NONE,
-                isSoftDeletesField);
-        fi =
-            new FieldInfo(
-                name,
-                fieldNumber,
-                false,
-                false,
-                false,
-                IndexOptions.NONE,
-                DocValuesType.NONE,
-                -1,
-                new HashMap<>(),
-                0,
-                0,
-                0,
-                0,
-                VectorValues.SearchStrategy.NONE,
-                isSoftDeletesField);
-        assert !byName.containsKey(fi.name);
-        globalFieldNumbers.verifyConsistent(
-            Integer.valueOf(fi.number), fi.name, DocValuesType.NONE);
-        byName.put(fi.name, fi);
+      if (fi != null) {
+        return fi;
+      } else {
+        return addField(
+            name,
+            -1,
+            false,
+            false,
+            false,
+            IndexOptions.NONE,
+            DocValuesType.NONE,
+            -1,
+            new HashMap<>(),
+            0,
+            0,
+            0,
+            0,
+            VectorValues.SearchStrategy.NONE,
+            name.equals(globalFieldNumbers.softDeletesFieldName));
       }
+    }
 
-      return fi;
+    public FieldInfo add(FieldInfo fi) {
+      return add(fi, -1);
+    }
+
+    public FieldInfo add(FieldInfo fi, long dvGen) {

Review comment:
       I could not remove `add(FieldInfo fi, long dvGen)` as `TestParallelLeafReader::testWithDocValuesUpdates()` [expects dvGen to be unchanged](https://github.com/apache/lucene/blob/main/lucene/core/src/test/org/apache/lucene/index/TestParallelLeafReader.java#L408).




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r598288400



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -972,6 +671,64 @@ private boolean assertNotFinished() {
       return true;
     }
 
+    private FieldInfo addField(
+        String name,
+        int preferredFieldNumber,
+        boolean storeTermVector,
+        boolean omitNorms,
+        boolean storePayloads,
+        IndexOptions indexOptions,
+        DocValuesType docValues,
+        long dvGen,
+        Map<String, String> attributes,
+        int dataDimensionCount,
+        int indexDimensionCount,
+        int dimensionNumBytes,
+        int vectorDimension,
+        VectorValues.SearchStrategy vectorSearchStrategy,
+        boolean isSoftDeletesField) {
+      // This field wasn't yet added to this in-RAM
+      // segment's FieldInfo, so now we get a global
+      // number for this field.  If the field was seen
+      // before then we'll get the same name and number,
+      // else we'll allocate a new one:
+      assert assertNotFinished();
+      final int fieldNumber =
+          globalFieldNumbers.addOrGet(
+              name,
+              preferredFieldNumber,
+              indexOptions,
+              storeTermVector,
+              omitNorms,
+              docValues,
+              dataDimensionCount,
+              indexDimensionCount,
+              dimensionNumBytes,
+              vectorDimension,
+              vectorSearchStrategy,
+              isSoftDeletesField);
+      FieldInfo fi =
+          new FieldInfo(
+              name,
+              fieldNumber,
+              storeTermVector,
+              omitNorms,
+              storePayloads,
+              indexOptions,
+              docValues,
+              dvGen,
+              attributes,
+              dataDimensionCount,
+              indexDimensionCount,
+              dimensionNumBytes,
+              vectorDimension,
+              vectorSearchStrategy,
+              isSoftDeletesField);
+      assert byName.containsKey(fi.name) == false;

Review comment:
       I've removed the assertion, as it is unnecessary.  Before it this function we already checked that `byName` doesn't contain `fi.name`, and since `FieldInfos.Builder` is intended to be used in a single thread, this extra check or assertion is unnecessary.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on pull request #11:
URL: https://github.com/apache/lucene/pull/11#issuecomment-800620410


   @s1monw  Thank you for your review. I will go through your comments, and will address them. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova merged pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova merged pull request #11:
URL: https://github.com/apache/lucene/pull/11


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r596378102



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -111,6 +111,43 @@ public FieldInfo(
 
   /** Performs internal consistency checks. Always returns true (or throws IllegalStateException) */
   public boolean checkConsistency() {
+    return checkOptionsCorrectness(
+        name,
+        storeTermVector,
+        omitNorms,
+        storePayloads,
+        indexOptions,
+        docValuesType,
+        dvGen,
+        pointDimensionCount,
+        pointIndexDimensionCount,
+        pointNumBytes,
+        vectorDimension,
+        vectorSearchStrategy);
+  }
+
+  /**
+   * Check correctness of FieldInfo options
+   *
+   * @throws IllegalStateException if some options are incorrect
+   * @return {@code true} if all options are correct
+   */
+  public static boolean checkOptionsCorrectness(
+      String name,
+      boolean storeTermVector,
+      boolean omitNorms,
+      boolean storePayloads,
+      IndexOptions indexOptions,
+      DocValuesType docValuesType,
+      long dvGen,
+      int pointDimensionCount,
+      int pointIndexDimensionCount,
+      int pointNumBytes,
+      int vectorDimension,
+      VectorValues.SearchStrategy vectorSearchStrategy) {
+    if (indexOptions == null) {
+      throw new IllegalStateException("IndexOptions must not be null (field: '" + name + "')");

Review comment:
       @s1monw  Do you mean a separate static method for each different error message? 
   In a follow-up PR I plan to remove set methods from FieldInfo (`setDocValuesType`,  `setIndexOptions`) where some of these error messages are repeated.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r596391496



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -796,167 +565,97 @@ public void add(FieldInfos other) {
     /** Create a new field, or return existing one. */
     public FieldInfo getOrAdd(String name) {
       FieldInfo fi = fieldInfo(name);
-      if (fi == null) {
-        assert assertNotFinished();
-        // This field wasn't yet added to this in-RAM
-        // segment's FieldInfo, so now we get a global
-        // number for this field.  If the field was seen
-        // before then we'll get the same name and number,
-        // else we'll allocate a new one:
-        final boolean isSoftDeletesField = name.equals(globalFieldNumbers.softDeletesFieldName);
-        final int fieldNumber =
-            globalFieldNumbers.addOrGet(
-                name,
-                -1,
-                IndexOptions.NONE,
-                DocValuesType.NONE,
-                0,
-                0,
-                0,
-                0,
-                VectorValues.SearchStrategy.NONE,
-                isSoftDeletesField);
-        fi =
-            new FieldInfo(
-                name,
-                fieldNumber,
-                false,
-                false,
-                false,
-                IndexOptions.NONE,
-                DocValuesType.NONE,
-                -1,
-                new HashMap<>(),
-                0,
-                0,
-                0,
-                0,
-                VectorValues.SearchStrategy.NONE,
-                isSoftDeletesField);
-        assert !byName.containsKey(fi.name);
-        globalFieldNumbers.verifyConsistent(
-            Integer.valueOf(fi.number), fi.name, DocValuesType.NONE);
-        byName.put(fi.name, fi);
+      if (fi != null) {
+        return fi;
+      } else {
+        return addField(
+            name,
+            -1,
+            false,
+            false,
+            false,
+            IndexOptions.NONE,
+            DocValuesType.NONE,
+            -1,
+            new HashMap<>(),
+            0,
+            0,
+            0,
+            0,
+            VectorValues.SearchStrategy.NONE,
+            name.equals(globalFieldNumbers.softDeletesFieldName));
       }
+    }
 
-      return fi;
+    public FieldInfo add(FieldInfo fi) {
+      return add(fi, -1);
+    }
+
+    public FieldInfo add(FieldInfo fi, long dvGen) {
+      // IMPORTANT - reuse the field number if possible for consistent field numbers across segments
+      if (fi.getDocValuesType() == null) {
+        throw new NullPointerException("DocValuesType must not be null");
+      }
+      final FieldInfo curFi = fieldInfo(fi.name);
+      if (curFi == null) {
+        // original attributes is UnmodifiableMap
+        Map<String, String> attributes =
+            fi.attributes() == null ? null : new HashMap<>(fi.attributes());
+        return addField(
+            fi.name,
+            fi.number,
+            fi.hasVectors(),
+            fi.omitsNorms(),
+            fi.hasPayloads(),
+            fi.getIndexOptions(),
+            fi.getDocValuesType(),
+            dvGen,
+            attributes,
+            fi.getPointDimensionCount(),
+            fi.getPointIndexDimensionCount(),
+            fi.getPointNumBytes(),
+            fi.getVectorDimension(),
+            fi.getVectorSearchStrategy(),
+            fi.isSoftDeletesField());
+      } else {
+        curFi.verifySameSchema(fi, dvGen);

Review comment:
       We indeed can use `equals` to compare, but then we would be able to return only a general error message without the ability to specify which specific options are  different. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r596388510



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -130,127 +167,252 @@ public boolean checkConsistency() {
       }
     }
 
-    if (pointDimensionCount < 0) {
+    if (docValuesType == null) {
+      throw new IllegalStateException("DocValuesType must not be null (field: '" + name + "')");
+    }
+    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
       throw new IllegalStateException(
-          "pointDimensionCount must be >= 0; got " + pointDimensionCount);
+          "field '"
+              + name
+              + "' cannot have a docvalues update generation without having docvalues");
     }
 
+    if (pointDimensionCount < 0) {
+      throw new IllegalStateException(
+          "pointDimensionCount must be >= 0; got "
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
+    }
     if (pointIndexDimensionCount < 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be >= 0; got " + pointIndexDimensionCount);
+          "pointIndexDimensionCount must be >= 0; got "
+              + pointIndexDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes < 0) {
-      throw new IllegalStateException("pointNumBytes must be >= 0; got " + pointNumBytes);
+      throw new IllegalStateException(
+          "pointNumBytes must be >= 0; got " + pointNumBytes + " (field: '" + name + "')");
     }
 
     if (pointDimensionCount != 0 && pointNumBytes == 0) {
       throw new IllegalStateException(
-          "pointNumBytes must be > 0 when pointDimensionCount=" + pointDimensionCount);
+          "pointNumBytes must be > 0 when pointDimensionCount="
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointIndexDimensionCount != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be 0 when pointDimensionCount=0");
+          "pointIndexDimensionCount must be 0 when pointDimensionCount=0"
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointDimensionCount must be > 0 when pointNumBytes=" + pointNumBytes);
+          "pointDimensionCount must be > 0 when pointNumBytes="
+              + pointNumBytes
+              + " (field: '"
+              + name
+              + "')");
     }
 
-    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
+    if (vectorSearchStrategy == null) {
       throw new IllegalStateException(
-          "field '"
-              + name
-              + "' cannot have a docvalues update generation without having docvalues");
+          "Vector search strategy must not be null (field: '" + name + "')");
     }
-
     if (vectorDimension < 0) {
-      throw new IllegalStateException("vectorDimension must be >=0; got " + vectorDimension);
+      throw new IllegalStateException(
+          "vectorDimension must be >=0; got " + vectorDimension + " (field: '" + name + "')");
     }
-
     if (vectorDimension == 0 && vectorSearchStrategy != VectorValues.SearchStrategy.NONE) {
       throw new IllegalStateException(
-          "vector search strategy must be NONE when dimension = 0; got " + vectorSearchStrategy);
+          "vector search strategy must be NONE when dimension = 0; got "
+              + vectorSearchStrategy
+              + " (field: '"
+              + name
+              + "')");
     }
-
     return true;
   }
 
-  // should only be called by FieldInfos#addOrUpdate
-  void update(
-      boolean storeTermVector,
-      boolean omitNorms,
-      boolean storePayloads,
-      IndexOptions indexOptions,
-      Map<String, String> attributes,
-      int dimensionCount,
-      int indexDimensionCount,
-      int dimensionNumBytes) {
-    if (indexOptions == null) {
-      throw new NullPointerException("IndexOptions must not be null (field: \"" + name + "\")");
-    }
-    // System.out.println("FI.update field=" + name + " indexed=" + indexed + " omitNorms=" +
-    // omitNorms + " this.omitNorms=" + this.omitNorms);
-    if (this.indexOptions != indexOptions) {
-      if (this.indexOptions == IndexOptions.NONE) {
-        this.indexOptions = indexOptions;
-      } else if (indexOptions != IndexOptions.NONE) {
-        throw new IllegalArgumentException(
-            "cannot change field \""
-                + name
-                + "\" from index options="
-                + this.indexOptions
-                + " to inconsistent index options="
-                + indexOptions);
-      }
+  void verifySameSchema(FieldInfo o, long dvGen) {
+    String fieldName = this.name;
+    verifySameIndexOptions(fieldName, this.indexOptions, o.getIndexOptions());
+    if (this.indexOptions != IndexOptions.NONE) {
+      verifySameOmitNorms(fieldName, this.omitNorms, o.omitNorms);
+      verifySameStoreTermVectors(fieldName, this.storeTermVector, o.storeTermVector);
+    }
+    verifySameDocValuesType(fieldName, this.docValuesType, o.docValuesType);
+    verifySameDVGen(fieldName, this.dvGen, dvGen);
+    verifySamePointsOptions(
+        fieldName,
+        this.pointDimensionCount,
+        this.pointIndexDimensionCount,
+        this.pointNumBytes,
+        o.pointDimensionCount,
+        o.pointIndexDimensionCount,
+        o.pointNumBytes);
+    verifySameVectorOptions(
+        fieldName,
+        this.vectorDimension,
+        this.vectorSearchStrategy,
+        o.vectorDimension,
+        o.vectorSearchStrategy);
+  }
+
+  /**
+   * Verify that the provided index options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameIndexOptions(
+      String fieldName, IndexOptions indexOptions1, IndexOptions indexOptions2) {
+    if (indexOptions1 != indexOptions2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from index options="
+              + indexOptions1
+              + " to inconsistent index options="
+              + indexOptions2);
     }
+  }
 
-    if (this.pointDimensionCount == 0 && dimensionCount != 0) {
-      this.pointDimensionCount = dimensionCount;
-      this.pointIndexDimensionCount = indexDimensionCount;
-      this.pointNumBytes = dimensionNumBytes;
-    } else if (dimensionCount != 0
-        && (this.pointDimensionCount != dimensionCount
-            || this.pointIndexDimensionCount != indexDimensionCount
-            || this.pointNumBytes != dimensionNumBytes)) {
+  /**
+   * Verify that the provided docValues type are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameDocValuesType(
+      String fieldName, DocValuesType docValuesType1, DocValuesType docValuesType2) {
+    if (docValuesType1 != docValuesType2) {
       throw new IllegalArgumentException(
           "cannot change field \""
-              + name
+              + fieldName
+              + "\" from doc values type="
+              + docValuesType1
+              + " to inconsistent doc values type="
+              + docValuesType2);
+    }
+  }
+
+  /**
+   * Verify that the provided doc values generations are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  // TODO: not sure if gen also must be the same
+  public static void verifySameDVGen(String fieldName, long docValuesGen1, long docValuesGen2) {
+    if (docValuesGen1 != docValuesGen2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from doc values generation="
+              + docValuesGen1
+              + " to inconsistent doc values generation="
+              + docValuesGen2);
+    }
+  }
+
+  /**
+   * Verify that the provided store term vectors options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameStoreTermVectors(

Review comment:
       Addressed in 0fe3493110ac2a5f750ad41f732436daff6c69f5

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -130,127 +167,252 @@ public boolean checkConsistency() {
       }
     }
 
-    if (pointDimensionCount < 0) {
+    if (docValuesType == null) {
+      throw new IllegalStateException("DocValuesType must not be null (field: '" + name + "')");
+    }
+    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
       throw new IllegalStateException(
-          "pointDimensionCount must be >= 0; got " + pointDimensionCount);
+          "field '"
+              + name
+              + "' cannot have a docvalues update generation without having docvalues");
     }
 
+    if (pointDimensionCount < 0) {
+      throw new IllegalStateException(
+          "pointDimensionCount must be >= 0; got "
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
+    }
     if (pointIndexDimensionCount < 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be >= 0; got " + pointIndexDimensionCount);
+          "pointIndexDimensionCount must be >= 0; got "
+              + pointIndexDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes < 0) {
-      throw new IllegalStateException("pointNumBytes must be >= 0; got " + pointNumBytes);
+      throw new IllegalStateException(
+          "pointNumBytes must be >= 0; got " + pointNumBytes + " (field: '" + name + "')");
     }
 
     if (pointDimensionCount != 0 && pointNumBytes == 0) {
       throw new IllegalStateException(
-          "pointNumBytes must be > 0 when pointDimensionCount=" + pointDimensionCount);
+          "pointNumBytes must be > 0 when pointDimensionCount="
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointIndexDimensionCount != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be 0 when pointDimensionCount=0");
+          "pointIndexDimensionCount must be 0 when pointDimensionCount=0"
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointDimensionCount must be > 0 when pointNumBytes=" + pointNumBytes);
+          "pointDimensionCount must be > 0 when pointNumBytes="
+              + pointNumBytes
+              + " (field: '"
+              + name
+              + "')");
     }
 
-    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
+    if (vectorSearchStrategy == null) {
       throw new IllegalStateException(
-          "field '"
-              + name
-              + "' cannot have a docvalues update generation without having docvalues");
+          "Vector search strategy must not be null (field: '" + name + "')");
     }
-
     if (vectorDimension < 0) {
-      throw new IllegalStateException("vectorDimension must be >=0; got " + vectorDimension);
+      throw new IllegalStateException(
+          "vectorDimension must be >=0; got " + vectorDimension + " (field: '" + name + "')");
     }
-
     if (vectorDimension == 0 && vectorSearchStrategy != VectorValues.SearchStrategy.NONE) {
       throw new IllegalStateException(
-          "vector search strategy must be NONE when dimension = 0; got " + vectorSearchStrategy);
+          "vector search strategy must be NONE when dimension = 0; got "
+              + vectorSearchStrategy
+              + " (field: '"
+              + name
+              + "')");
     }
-
     return true;
   }
 
-  // should only be called by FieldInfos#addOrUpdate
-  void update(
-      boolean storeTermVector,
-      boolean omitNorms,
-      boolean storePayloads,
-      IndexOptions indexOptions,
-      Map<String, String> attributes,
-      int dimensionCount,
-      int indexDimensionCount,
-      int dimensionNumBytes) {
-    if (indexOptions == null) {
-      throw new NullPointerException("IndexOptions must not be null (field: \"" + name + "\")");
-    }
-    // System.out.println("FI.update field=" + name + " indexed=" + indexed + " omitNorms=" +
-    // omitNorms + " this.omitNorms=" + this.omitNorms);
-    if (this.indexOptions != indexOptions) {
-      if (this.indexOptions == IndexOptions.NONE) {
-        this.indexOptions = indexOptions;
-      } else if (indexOptions != IndexOptions.NONE) {
-        throw new IllegalArgumentException(
-            "cannot change field \""
-                + name
-                + "\" from index options="
-                + this.indexOptions
-                + " to inconsistent index options="
-                + indexOptions);
-      }
+  void verifySameSchema(FieldInfo o, long dvGen) {
+    String fieldName = this.name;
+    verifySameIndexOptions(fieldName, this.indexOptions, o.getIndexOptions());
+    if (this.indexOptions != IndexOptions.NONE) {
+      verifySameOmitNorms(fieldName, this.omitNorms, o.omitNorms);
+      verifySameStoreTermVectors(fieldName, this.storeTermVector, o.storeTermVector);
+    }
+    verifySameDocValuesType(fieldName, this.docValuesType, o.docValuesType);
+    verifySameDVGen(fieldName, this.dvGen, dvGen);
+    verifySamePointsOptions(
+        fieldName,
+        this.pointDimensionCount,
+        this.pointIndexDimensionCount,
+        this.pointNumBytes,
+        o.pointDimensionCount,
+        o.pointIndexDimensionCount,
+        o.pointNumBytes);
+    verifySameVectorOptions(
+        fieldName,
+        this.vectorDimension,
+        this.vectorSearchStrategy,
+        o.vectorDimension,
+        o.vectorSearchStrategy);
+  }
+
+  /**
+   * Verify that the provided index options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameIndexOptions(
+      String fieldName, IndexOptions indexOptions1, IndexOptions indexOptions2) {
+    if (indexOptions1 != indexOptions2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from index options="
+              + indexOptions1
+              + " to inconsistent index options="
+              + indexOptions2);
     }
+  }
 
-    if (this.pointDimensionCount == 0 && dimensionCount != 0) {
-      this.pointDimensionCount = dimensionCount;
-      this.pointIndexDimensionCount = indexDimensionCount;
-      this.pointNumBytes = dimensionNumBytes;
-    } else if (dimensionCount != 0
-        && (this.pointDimensionCount != dimensionCount
-            || this.pointIndexDimensionCount != indexDimensionCount
-            || this.pointNumBytes != dimensionNumBytes)) {
+  /**
+   * Verify that the provided docValues type are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameDocValuesType(
+      String fieldName, DocValuesType docValuesType1, DocValuesType docValuesType2) {
+    if (docValuesType1 != docValuesType2) {
       throw new IllegalArgumentException(
           "cannot change field \""
-              + name
+              + fieldName
+              + "\" from doc values type="
+              + docValuesType1
+              + " to inconsistent doc values type="
+              + docValuesType2);
+    }
+  }
+
+  /**
+   * Verify that the provided doc values generations are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  // TODO: not sure if gen also must be the same
+  public static void verifySameDVGen(String fieldName, long docValuesGen1, long docValuesGen2) {

Review comment:
       Addressed in 0fe3493110ac2a5f750ad41f732436daff6c69f5




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] s1monw commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
s1monw commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r593002781



##########
File path: lucene/MIGRATE.md
##########
@@ -358,8 +358,10 @@ Rather, an IllegalArgumentException shall be thrown. This is introduced for bett
 defence and to ensure that there is no bubbling up of errors when Lucene is
 used in multi level applications
 
-## Assumption of data consistency between different data-structures sharing the same field name
+### Require consistency between data-structures on a per-field basis
 
-Sorting on a numeric field that is indexed with both doc values and points may use an
-optimization to skip non-competitive documents. This optimization relies on the assumption
-that the same data is stored in these points and doc values.
+A field must be indexed with the same index options and data-structures across 
+all documents within an index. Thus, for example, it is not allowed to have 
+one document in a index where a certain field is indexed with doc values 
+and points, and another document where the same field is indexed only with 
+points.

Review comment:
       would it make sense to say something like "the per field data-structures are implicitly defined by the first document indexed that contains a certain field"?

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -130,127 +167,252 @@ public boolean checkConsistency() {
       }
     }
 
-    if (pointDimensionCount < 0) {
+    if (docValuesType == null) {
+      throw new IllegalStateException("DocValuesType must not be null (field: '" + name + "')");

Review comment:
       same here? IAE?

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -130,127 +167,252 @@ public boolean checkConsistency() {
       }
     }
 
-    if (pointDimensionCount < 0) {
+    if (docValuesType == null) {
+      throw new IllegalStateException("DocValuesType must not be null (field: '" + name + "')");
+    }
+    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
       throw new IllegalStateException(
-          "pointDimensionCount must be >= 0; got " + pointDimensionCount);
+          "field '"
+              + name
+              + "' cannot have a docvalues update generation without having docvalues");
     }
 
+    if (pointDimensionCount < 0) {
+      throw new IllegalStateException(
+          "pointDimensionCount must be >= 0; got "
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
+    }
     if (pointIndexDimensionCount < 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be >= 0; got " + pointIndexDimensionCount);

Review comment:
       do we need reformat this? Can it be on the same line?

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -130,127 +167,252 @@ public boolean checkConsistency() {
       }
     }
 
-    if (pointDimensionCount < 0) {
+    if (docValuesType == null) {
+      throw new IllegalStateException("DocValuesType must not be null (field: '" + name + "')");
+    }
+    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
       throw new IllegalStateException(
-          "pointDimensionCount must be >= 0; got " + pointDimensionCount);
+          "field '"
+              + name
+              + "' cannot have a docvalues update generation without having docvalues");
     }
 
+    if (pointDimensionCount < 0) {
+      throw new IllegalStateException(
+          "pointDimensionCount must be >= 0; got "
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
+    }
     if (pointIndexDimensionCount < 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be >= 0; got " + pointIndexDimensionCount);
+          "pointIndexDimensionCount must be >= 0; got "
+              + pointIndexDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes < 0) {
-      throw new IllegalStateException("pointNumBytes must be >= 0; got " + pointNumBytes);
+      throw new IllegalStateException(
+          "pointNumBytes must be >= 0; got " + pointNumBytes + " (field: '" + name + "')");
     }
 
     if (pointDimensionCount != 0 && pointNumBytes == 0) {
       throw new IllegalStateException(
-          "pointNumBytes must be > 0 when pointDimensionCount=" + pointDimensionCount);
+          "pointNumBytes must be > 0 when pointDimensionCount="
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointIndexDimensionCount != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be 0 when pointDimensionCount=0");
+          "pointIndexDimensionCount must be 0 when pointDimensionCount=0"
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointDimensionCount must be > 0 when pointNumBytes=" + pointNumBytes);
+          "pointDimensionCount must be > 0 when pointNumBytes="
+              + pointNumBytes
+              + " (field: '"
+              + name
+              + "')");
     }
 
-    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
+    if (vectorSearchStrategy == null) {
       throw new IllegalStateException(
-          "field '"
-              + name
-              + "' cannot have a docvalues update generation without having docvalues");
+          "Vector search strategy must not be null (field: '" + name + "')");
     }
-
     if (vectorDimension < 0) {
-      throw new IllegalStateException("vectorDimension must be >=0; got " + vectorDimension);
+      throw new IllegalStateException(
+          "vectorDimension must be >=0; got " + vectorDimension + " (field: '" + name + "')");
     }
-
     if (vectorDimension == 0 && vectorSearchStrategy != VectorValues.SearchStrategy.NONE) {
       throw new IllegalStateException(
-          "vector search strategy must be NONE when dimension = 0; got " + vectorSearchStrategy);
+          "vector search strategy must be NONE when dimension = 0; got "
+              + vectorSearchStrategy
+              + " (field: '"
+              + name
+              + "')");
     }
-
     return true;
   }
 
-  // should only be called by FieldInfos#addOrUpdate
-  void update(
-      boolean storeTermVector,
-      boolean omitNorms,
-      boolean storePayloads,
-      IndexOptions indexOptions,
-      Map<String, String> attributes,
-      int dimensionCount,
-      int indexDimensionCount,
-      int dimensionNumBytes) {
-    if (indexOptions == null) {
-      throw new NullPointerException("IndexOptions must not be null (field: \"" + name + "\")");
-    }
-    // System.out.println("FI.update field=" + name + " indexed=" + indexed + " omitNorms=" +
-    // omitNorms + " this.omitNorms=" + this.omitNorms);
-    if (this.indexOptions != indexOptions) {
-      if (this.indexOptions == IndexOptions.NONE) {
-        this.indexOptions = indexOptions;
-      } else if (indexOptions != IndexOptions.NONE) {
-        throw new IllegalArgumentException(
-            "cannot change field \""
-                + name
-                + "\" from index options="
-                + this.indexOptions
-                + " to inconsistent index options="
-                + indexOptions);
-      }
+  void verifySameSchema(FieldInfo o, long dvGen) {
+    String fieldName = this.name;
+    verifySameIndexOptions(fieldName, this.indexOptions, o.getIndexOptions());
+    if (this.indexOptions != IndexOptions.NONE) {
+      verifySameOmitNorms(fieldName, this.omitNorms, o.omitNorms);
+      verifySameStoreTermVectors(fieldName, this.storeTermVector, o.storeTermVector);
+    }
+    verifySameDocValuesType(fieldName, this.docValuesType, o.docValuesType);
+    verifySameDVGen(fieldName, this.dvGen, dvGen);
+    verifySamePointsOptions(
+        fieldName,
+        this.pointDimensionCount,
+        this.pointIndexDimensionCount,
+        this.pointNumBytes,
+        o.pointDimensionCount,
+        o.pointIndexDimensionCount,
+        o.pointNumBytes);
+    verifySameVectorOptions(
+        fieldName,
+        this.vectorDimension,
+        this.vectorSearchStrategy,
+        o.vectorDimension,
+        o.vectorSearchStrategy);
+  }
+
+  /**
+   * Verify that the provided index options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameIndexOptions(
+      String fieldName, IndexOptions indexOptions1, IndexOptions indexOptions2) {
+    if (indexOptions1 != indexOptions2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from index options="
+              + indexOptions1
+              + " to inconsistent index options="
+              + indexOptions2);
     }
+  }
 
-    if (this.pointDimensionCount == 0 && dimensionCount != 0) {
-      this.pointDimensionCount = dimensionCount;
-      this.pointIndexDimensionCount = indexDimensionCount;
-      this.pointNumBytes = dimensionNumBytes;
-    } else if (dimensionCount != 0
-        && (this.pointDimensionCount != dimensionCount
-            || this.pointIndexDimensionCount != indexDimensionCount
-            || this.pointNumBytes != dimensionNumBytes)) {
+  /**
+   * Verify that the provided docValues type are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameDocValuesType(
+      String fieldName, DocValuesType docValuesType1, DocValuesType docValuesType2) {
+    if (docValuesType1 != docValuesType2) {
       throw new IllegalArgumentException(
           "cannot change field \""
-              + name
+              + fieldName
+              + "\" from doc values type="
+              + docValuesType1
+              + " to inconsistent doc values type="
+              + docValuesType2);
+    }
+  }
+
+  /**
+   * Verify that the provided doc values generations are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  // TODO: not sure if gen also must be the same
+  public static void verifySameDVGen(String fieldName, long docValuesGen1, long docValuesGen2) {

Review comment:
       can be pkg private?

##########
File path: lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java
##########
@@ -149,9 +149,6 @@ boolean start(IndexableField field, boolean first) {
       doVectors = field.fieldType().storeTermVectors();
 
       if (doVectors) {
-
-        termsWriter.hasVectors = true;

Review comment:
       we don't set this here anymore? why not?

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -339,110 +350,17 @@ synchronized int addOrGet(
         String fieldName,
         int preferredFieldNumber,
         IndexOptions indexOptions,
+        boolean storeTermVector,

Review comment:
       we duplicate the args list from FieldInfo across many files. I wonder if we should rather go and pay the price of constructing more than one FieldInfo object in some cases and trade this for calling a simpler method in many places. For instance everywhere where we pass all these args we can pass a FieldInfo object instead. In this method we can modify it if we can't assign the preferred FN or if it's `-1` WDYT?

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -130,127 +167,252 @@ public boolean checkConsistency() {
       }
     }
 
-    if (pointDimensionCount < 0) {
+    if (docValuesType == null) {
+      throw new IllegalStateException("DocValuesType must not be null (field: '" + name + "')");
+    }
+    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
       throw new IllegalStateException(
-          "pointDimensionCount must be >= 0; got " + pointDimensionCount);
+          "field '"
+              + name
+              + "' cannot have a docvalues update generation without having docvalues");
     }
 
+    if (pointDimensionCount < 0) {
+      throw new IllegalStateException(
+          "pointDimensionCount must be >= 0; got "
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
+    }
     if (pointIndexDimensionCount < 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be >= 0; got " + pointIndexDimensionCount);
+          "pointIndexDimensionCount must be >= 0; got "
+              + pointIndexDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes < 0) {
-      throw new IllegalStateException("pointNumBytes must be >= 0; got " + pointNumBytes);
+      throw new IllegalStateException(
+          "pointNumBytes must be >= 0; got " + pointNumBytes + " (field: '" + name + "')");
     }
 
     if (pointDimensionCount != 0 && pointNumBytes == 0) {
       throw new IllegalStateException(
-          "pointNumBytes must be > 0 when pointDimensionCount=" + pointDimensionCount);
+          "pointNumBytes must be > 0 when pointDimensionCount="
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointIndexDimensionCount != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be 0 when pointDimensionCount=0");
+          "pointIndexDimensionCount must be 0 when pointDimensionCount=0"
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointDimensionCount must be > 0 when pointNumBytes=" + pointNumBytes);
+          "pointDimensionCount must be > 0 when pointNumBytes="
+              + pointNumBytes
+              + " (field: '"
+              + name
+              + "')");
     }
 
-    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
+    if (vectorSearchStrategy == null) {
       throw new IllegalStateException(
-          "field '"
-              + name
-              + "' cannot have a docvalues update generation without having docvalues");
+          "Vector search strategy must not be null (field: '" + name + "')");
     }
-
     if (vectorDimension < 0) {
-      throw new IllegalStateException("vectorDimension must be >=0; got " + vectorDimension);
+      throw new IllegalStateException(
+          "vectorDimension must be >=0; got " + vectorDimension + " (field: '" + name + "')");
     }
-
     if (vectorDimension == 0 && vectorSearchStrategy != VectorValues.SearchStrategy.NONE) {
       throw new IllegalStateException(
-          "vector search strategy must be NONE when dimension = 0; got " + vectorSearchStrategy);
+          "vector search strategy must be NONE when dimension = 0; got "
+              + vectorSearchStrategy
+              + " (field: '"
+              + name
+              + "')");
     }
-
     return true;
   }
 
-  // should only be called by FieldInfos#addOrUpdate
-  void update(
-      boolean storeTermVector,
-      boolean omitNorms,
-      boolean storePayloads,
-      IndexOptions indexOptions,
-      Map<String, String> attributes,
-      int dimensionCount,
-      int indexDimensionCount,
-      int dimensionNumBytes) {
-    if (indexOptions == null) {
-      throw new NullPointerException("IndexOptions must not be null (field: \"" + name + "\")");
-    }
-    // System.out.println("FI.update field=" + name + " indexed=" + indexed + " omitNorms=" +
-    // omitNorms + " this.omitNorms=" + this.omitNorms);
-    if (this.indexOptions != indexOptions) {
-      if (this.indexOptions == IndexOptions.NONE) {
-        this.indexOptions = indexOptions;
-      } else if (indexOptions != IndexOptions.NONE) {
-        throw new IllegalArgumentException(
-            "cannot change field \""
-                + name
-                + "\" from index options="
-                + this.indexOptions
-                + " to inconsistent index options="
-                + indexOptions);
-      }
+  void verifySameSchema(FieldInfo o, long dvGen) {
+    String fieldName = this.name;
+    verifySameIndexOptions(fieldName, this.indexOptions, o.getIndexOptions());
+    if (this.indexOptions != IndexOptions.NONE) {
+      verifySameOmitNorms(fieldName, this.omitNorms, o.omitNorms);
+      verifySameStoreTermVectors(fieldName, this.storeTermVector, o.storeTermVector);
+    }
+    verifySameDocValuesType(fieldName, this.docValuesType, o.docValuesType);
+    verifySameDVGen(fieldName, this.dvGen, dvGen);
+    verifySamePointsOptions(
+        fieldName,
+        this.pointDimensionCount,
+        this.pointIndexDimensionCount,
+        this.pointNumBytes,
+        o.pointDimensionCount,
+        o.pointIndexDimensionCount,
+        o.pointNumBytes);
+    verifySameVectorOptions(
+        fieldName,
+        this.vectorDimension,
+        this.vectorSearchStrategy,
+        o.vectorDimension,
+        o.vectorSearchStrategy);
+  }
+
+  /**
+   * Verify that the provided index options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameIndexOptions(
+      String fieldName, IndexOptions indexOptions1, IndexOptions indexOptions2) {
+    if (indexOptions1 != indexOptions2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from index options="
+              + indexOptions1
+              + " to inconsistent index options="
+              + indexOptions2);
     }
+  }
 
-    if (this.pointDimensionCount == 0 && dimensionCount != 0) {
-      this.pointDimensionCount = dimensionCount;
-      this.pointIndexDimensionCount = indexDimensionCount;
-      this.pointNumBytes = dimensionNumBytes;
-    } else if (dimensionCount != 0
-        && (this.pointDimensionCount != dimensionCount
-            || this.pointIndexDimensionCount != indexDimensionCount
-            || this.pointNumBytes != dimensionNumBytes)) {
+  /**
+   * Verify that the provided docValues type are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameDocValuesType(
+      String fieldName, DocValuesType docValuesType1, DocValuesType docValuesType2) {
+    if (docValuesType1 != docValuesType2) {
       throw new IllegalArgumentException(
           "cannot change field \""
-              + name
+              + fieldName
+              + "\" from doc values type="
+              + docValuesType1
+              + " to inconsistent doc values type="
+              + docValuesType2);
+    }
+  }
+
+  /**
+   * Verify that the provided doc values generations are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  // TODO: not sure if gen also must be the same
+  public static void verifySameDVGen(String fieldName, long docValuesGen1, long docValuesGen2) {
+    if (docValuesGen1 != docValuesGen2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from doc values generation="
+              + docValuesGen1
+              + " to inconsistent doc values generation="
+              + docValuesGen2);
+    }
+  }
+
+  /**
+   * Verify that the provided store term vectors options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameStoreTermVectors(

Review comment:
       can this be pkg protected?

##########
File path: lucene/CHANGES.txt
##########
@@ -92,6 +92,12 @@ API Changes
 * LUCENE-9480: Make DataInput's skipBytes(long) abstract as the implementation was not performant.
   IndexInput's api is unaffected: skipBytes() is implemented via seek(). (Greg Miller)
 
+* LUCENE-9334:  Require consistency between data-structures on a per-field basis.
+  A field across all documents within an index must be indexed with the same index
+  options and data-structures. As a consequence of this, doc values updates are
+  only applicable for fields that are indexed with doc values only

Review comment:
       this needs your name on it in parentheses ;) 

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -111,6 +111,43 @@ public FieldInfo(
 
   /** Performs internal consistency checks. Always returns true (or throws IllegalStateException) */
   public boolean checkConsistency() {
+    return checkOptionsCorrectness(
+        name,
+        storeTermVector,
+        omitNorms,
+        storePayloads,
+        indexOptions,
+        docValuesType,
+        dvGen,
+        pointDimensionCount,
+        pointIndexDimensionCount,
+        pointNumBytes,
+        vectorDimension,
+        vectorSearchStrategy);
+  }
+
+  /**
+   * Check correctness of FieldInfo options
+   *
+   * @throws IllegalStateException if some options are incorrect
+   * @return {@code true} if all options are correct
+   */
+  public static boolean checkOptionsCorrectness(
+      String name,

Review comment:
       is it required to wrap around every argument? I'd prefer to not do that it's hard to read IMO

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -130,127 +167,252 @@ public boolean checkConsistency() {
       }
     }
 
-    if (pointDimensionCount < 0) {
+    if (docValuesType == null) {
+      throw new IllegalStateException("DocValuesType must not be null (field: '" + name + "')");
+    }
+    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
       throw new IllegalStateException(
-          "pointDimensionCount must be >= 0; got " + pointDimensionCount);
+          "field '"
+              + name
+              + "' cannot have a docvalues update generation without having docvalues");
     }
 
+    if (pointDimensionCount < 0) {
+      throw new IllegalStateException(
+          "pointDimensionCount must be >= 0; got "
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
+    }
     if (pointIndexDimensionCount < 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be >= 0; got " + pointIndexDimensionCount);
+          "pointIndexDimensionCount must be >= 0; got "
+              + pointIndexDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes < 0) {
-      throw new IllegalStateException("pointNumBytes must be >= 0; got " + pointNumBytes);
+      throw new IllegalStateException(
+          "pointNumBytes must be >= 0; got " + pointNumBytes + " (field: '" + name + "')");
     }
 
     if (pointDimensionCount != 0 && pointNumBytes == 0) {
       throw new IllegalStateException(
-          "pointNumBytes must be > 0 when pointDimensionCount=" + pointDimensionCount);
+          "pointNumBytes must be > 0 when pointDimensionCount="
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointIndexDimensionCount != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be 0 when pointDimensionCount=0");
+          "pointIndexDimensionCount must be 0 when pointDimensionCount=0"
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointDimensionCount must be > 0 when pointNumBytes=" + pointNumBytes);
+          "pointDimensionCount must be > 0 when pointNumBytes="
+              + pointNumBytes
+              + " (field: '"
+              + name
+              + "')");
     }
 
-    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
+    if (vectorSearchStrategy == null) {
       throw new IllegalStateException(
-          "field '"
-              + name
-              + "' cannot have a docvalues update generation without having docvalues");
+          "Vector search strategy must not be null (field: '" + name + "')");
     }
-
     if (vectorDimension < 0) {
-      throw new IllegalStateException("vectorDimension must be >=0; got " + vectorDimension);
+      throw new IllegalStateException(
+          "vectorDimension must be >=0; got " + vectorDimension + " (field: '" + name + "')");
     }
-
     if (vectorDimension == 0 && vectorSearchStrategy != VectorValues.SearchStrategy.NONE) {
       throw new IllegalStateException(
-          "vector search strategy must be NONE when dimension = 0; got " + vectorSearchStrategy);
+          "vector search strategy must be NONE when dimension = 0; got "
+              + vectorSearchStrategy
+              + " (field: '"
+              + name
+              + "')");
     }
-
     return true;
   }
 
-  // should only be called by FieldInfos#addOrUpdate
-  void update(
-      boolean storeTermVector,
-      boolean omitNorms,
-      boolean storePayloads,
-      IndexOptions indexOptions,
-      Map<String, String> attributes,
-      int dimensionCount,
-      int indexDimensionCount,
-      int dimensionNumBytes) {
-    if (indexOptions == null) {
-      throw new NullPointerException("IndexOptions must not be null (field: \"" + name + "\")");
-    }
-    // System.out.println("FI.update field=" + name + " indexed=" + indexed + " omitNorms=" +
-    // omitNorms + " this.omitNorms=" + this.omitNorms);
-    if (this.indexOptions != indexOptions) {
-      if (this.indexOptions == IndexOptions.NONE) {
-        this.indexOptions = indexOptions;
-      } else if (indexOptions != IndexOptions.NONE) {
-        throw new IllegalArgumentException(
-            "cannot change field \""
-                + name
-                + "\" from index options="
-                + this.indexOptions
-                + " to inconsistent index options="
-                + indexOptions);
-      }
+  void verifySameSchema(FieldInfo o, long dvGen) {
+    String fieldName = this.name;
+    verifySameIndexOptions(fieldName, this.indexOptions, o.getIndexOptions());
+    if (this.indexOptions != IndexOptions.NONE) {
+      verifySameOmitNorms(fieldName, this.omitNorms, o.omitNorms);
+      verifySameStoreTermVectors(fieldName, this.storeTermVector, o.storeTermVector);
+    }
+    verifySameDocValuesType(fieldName, this.docValuesType, o.docValuesType);
+    verifySameDVGen(fieldName, this.dvGen, dvGen);
+    verifySamePointsOptions(
+        fieldName,
+        this.pointDimensionCount,
+        this.pointIndexDimensionCount,
+        this.pointNumBytes,
+        o.pointDimensionCount,
+        o.pointIndexDimensionCount,
+        o.pointNumBytes);
+    verifySameVectorOptions(
+        fieldName,
+        this.vectorDimension,
+        this.vectorSearchStrategy,
+        o.vectorDimension,
+        o.vectorSearchStrategy);
+  }
+
+  /**
+   * Verify that the provided index options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameIndexOptions(
+      String fieldName, IndexOptions indexOptions1, IndexOptions indexOptions2) {
+    if (indexOptions1 != indexOptions2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from index options="
+              + indexOptions1
+              + " to inconsistent index options="
+              + indexOptions2);
     }
+  }
 
-    if (this.pointDimensionCount == 0 && dimensionCount != 0) {
-      this.pointDimensionCount = dimensionCount;
-      this.pointIndexDimensionCount = indexDimensionCount;
-      this.pointNumBytes = dimensionNumBytes;
-    } else if (dimensionCount != 0
-        && (this.pointDimensionCount != dimensionCount
-            || this.pointIndexDimensionCount != indexDimensionCount
-            || this.pointNumBytes != dimensionNumBytes)) {
+  /**
+   * Verify that the provided docValues type are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameDocValuesType(
+      String fieldName, DocValuesType docValuesType1, DocValuesType docValuesType2) {
+    if (docValuesType1 != docValuesType2) {
       throw new IllegalArgumentException(
           "cannot change field \""
-              + name
+              + fieldName
+              + "\" from doc values type="
+              + docValuesType1
+              + " to inconsistent doc values type="
+              + docValuesType2);
+    }
+  }
+
+  /**
+   * Verify that the provided doc values generations are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  // TODO: not sure if gen also must be the same
+  public static void verifySameDVGen(String fieldName, long docValuesGen1, long docValuesGen2) {
+    if (docValuesGen1 != docValuesGen2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from doc values generation="
+              + docValuesGen1
+              + " to inconsistent doc values generation="
+              + docValuesGen2);
+    }
+  }
+
+  /**
+   * Verify that the provided store term vectors options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameStoreTermVectors(
+      String fieldName, boolean storeTermVector1, boolean storeTermVector2) {
+    if (storeTermVector1 != storeTermVector2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from storeTermVector="
+              + storeTermVector1
+              + " to inconsistent storeTermVector="
+              + storeTermVector2);
+    }
+  }
+
+  /**
+   * Verify that the provided omitNorms are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameOmitNorms(String fieldName, boolean omitNorms1, boolean omitNorms2) {
+    if (omitNorms1 != omitNorms2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from omitNorms="
+              + omitNorms1
+              + " to inconsistent omitNorms="
+              + omitNorms2);
+    }
+  }
+
+  /**
+   * Verify that the provided points indexing options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySamePointsOptions(

Review comment:
       can be pkg private?

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -130,127 +167,252 @@ public boolean checkConsistency() {
       }
     }
 
-    if (pointDimensionCount < 0) {
+    if (docValuesType == null) {
+      throw new IllegalStateException("DocValuesType must not be null (field: '" + name + "')");
+    }
+    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
       throw new IllegalStateException(
-          "pointDimensionCount must be >= 0; got " + pointDimensionCount);
+          "field '"
+              + name
+              + "' cannot have a docvalues update generation without having docvalues");
     }
 
+    if (pointDimensionCount < 0) {
+      throw new IllegalStateException(
+          "pointDimensionCount must be >= 0; got "
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
+    }
     if (pointIndexDimensionCount < 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be >= 0; got " + pointIndexDimensionCount);
+          "pointIndexDimensionCount must be >= 0; got "
+              + pointIndexDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes < 0) {
-      throw new IllegalStateException("pointNumBytes must be >= 0; got " + pointNumBytes);
+      throw new IllegalStateException(
+          "pointNumBytes must be >= 0; got " + pointNumBytes + " (field: '" + name + "')");
     }
 
     if (pointDimensionCount != 0 && pointNumBytes == 0) {
       throw new IllegalStateException(
-          "pointNumBytes must be > 0 when pointDimensionCount=" + pointDimensionCount);
+          "pointNumBytes must be > 0 when pointDimensionCount="
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointIndexDimensionCount != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be 0 when pointDimensionCount=0");
+          "pointIndexDimensionCount must be 0 when pointDimensionCount=0"
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointDimensionCount must be > 0 when pointNumBytes=" + pointNumBytes);
+          "pointDimensionCount must be > 0 when pointNumBytes="
+              + pointNumBytes
+              + " (field: '"
+              + name
+              + "')");
     }
 
-    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
+    if (vectorSearchStrategy == null) {
       throw new IllegalStateException(
-          "field '"
-              + name
-              + "' cannot have a docvalues update generation without having docvalues");
+          "Vector search strategy must not be null (field: '" + name + "')");
     }
-
     if (vectorDimension < 0) {
-      throw new IllegalStateException("vectorDimension must be >=0; got " + vectorDimension);
+      throw new IllegalStateException(
+          "vectorDimension must be >=0; got " + vectorDimension + " (field: '" + name + "')");
     }
-
     if (vectorDimension == 0 && vectorSearchStrategy != VectorValues.SearchStrategy.NONE) {
       throw new IllegalStateException(
-          "vector search strategy must be NONE when dimension = 0; got " + vectorSearchStrategy);
+          "vector search strategy must be NONE when dimension = 0; got "
+              + vectorSearchStrategy
+              + " (field: '"
+              + name
+              + "')");
     }
-
     return true;
   }
 
-  // should only be called by FieldInfos#addOrUpdate
-  void update(
-      boolean storeTermVector,
-      boolean omitNorms,
-      boolean storePayloads,
-      IndexOptions indexOptions,
-      Map<String, String> attributes,
-      int dimensionCount,
-      int indexDimensionCount,
-      int dimensionNumBytes) {
-    if (indexOptions == null) {
-      throw new NullPointerException("IndexOptions must not be null (field: \"" + name + "\")");
-    }
-    // System.out.println("FI.update field=" + name + " indexed=" + indexed + " omitNorms=" +
-    // omitNorms + " this.omitNorms=" + this.omitNorms);
-    if (this.indexOptions != indexOptions) {
-      if (this.indexOptions == IndexOptions.NONE) {
-        this.indexOptions = indexOptions;
-      } else if (indexOptions != IndexOptions.NONE) {
-        throw new IllegalArgumentException(
-            "cannot change field \""
-                + name
-                + "\" from index options="
-                + this.indexOptions
-                + " to inconsistent index options="
-                + indexOptions);
-      }
+  void verifySameSchema(FieldInfo o, long dvGen) {
+    String fieldName = this.name;
+    verifySameIndexOptions(fieldName, this.indexOptions, o.getIndexOptions());
+    if (this.indexOptions != IndexOptions.NONE) {
+      verifySameOmitNorms(fieldName, this.omitNorms, o.omitNorms);
+      verifySameStoreTermVectors(fieldName, this.storeTermVector, o.storeTermVector);
+    }
+    verifySameDocValuesType(fieldName, this.docValuesType, o.docValuesType);
+    verifySameDVGen(fieldName, this.dvGen, dvGen);
+    verifySamePointsOptions(
+        fieldName,
+        this.pointDimensionCount,
+        this.pointIndexDimensionCount,
+        this.pointNumBytes,
+        o.pointDimensionCount,
+        o.pointIndexDimensionCount,
+        o.pointNumBytes);
+    verifySameVectorOptions(
+        fieldName,
+        this.vectorDimension,
+        this.vectorSearchStrategy,
+        o.vectorDimension,
+        o.vectorSearchStrategy);
+  }
+
+  /**
+   * Verify that the provided index options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameIndexOptions(
+      String fieldName, IndexOptions indexOptions1, IndexOptions indexOptions2) {
+    if (indexOptions1 != indexOptions2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from index options="
+              + indexOptions1
+              + " to inconsistent index options="
+              + indexOptions2);
     }
+  }
 
-    if (this.pointDimensionCount == 0 && dimensionCount != 0) {
-      this.pointDimensionCount = dimensionCount;
-      this.pointIndexDimensionCount = indexDimensionCount;
-      this.pointNumBytes = dimensionNumBytes;
-    } else if (dimensionCount != 0
-        && (this.pointDimensionCount != dimensionCount
-            || this.pointIndexDimensionCount != indexDimensionCount
-            || this.pointNumBytes != dimensionNumBytes)) {
+  /**
+   * Verify that the provided docValues type are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameDocValuesType(

Review comment:
       can be pkg private?

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -111,6 +111,43 @@ public FieldInfo(
 
   /** Performs internal consistency checks. Always returns true (or throws IllegalStateException) */
   public boolean checkConsistency() {
+    return checkOptionsCorrectness(
+        name,
+        storeTermVector,
+        omitNorms,
+        storePayloads,
+        indexOptions,
+        docValuesType,
+        dvGen,
+        pointDimensionCount,
+        pointIndexDimensionCount,
+        pointNumBytes,
+        vectorDimension,
+        vectorSearchStrategy);
+  }
+
+  /**
+   * Check correctness of FieldInfo options
+   *
+   * @throws IllegalStateException if some options are incorrect
+   * @return {@code true} if all options are correct
+   */
+  public static boolean checkOptionsCorrectness(
+      String name,
+      boolean storeTermVector,
+      boolean omitNorms,
+      boolean storePayloads,
+      IndexOptions indexOptions,
+      DocValuesType docValuesType,
+      long dvGen,
+      int pointDimensionCount,
+      int pointIndexDimensionCount,
+      int pointNumBytes,
+      int vectorDimension,
+      VectorValues.SearchStrategy vectorSearchStrategy) {
+    if (indexOptions == null) {
+      throw new IllegalStateException("IndexOptions must not be null (field: '" + name + "')");

Review comment:
       should this be IAE instead?

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -130,127 +167,252 @@ public boolean checkConsistency() {
       }
     }
 
-    if (pointDimensionCount < 0) {
+    if (docValuesType == null) {
+      throw new IllegalStateException("DocValuesType must not be null (field: '" + name + "')");
+    }
+    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
       throw new IllegalStateException(
-          "pointDimensionCount must be >= 0; got " + pointDimensionCount);
+          "field '"
+              + name
+              + "' cannot have a docvalues update generation without having docvalues");
     }
 
+    if (pointDimensionCount < 0) {
+      throw new IllegalStateException(
+          "pointDimensionCount must be >= 0; got "

Review comment:
       this is preexisting right? I personally still thing IAE is better here. but I am ok both ways

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -130,127 +167,252 @@ public boolean checkConsistency() {
       }
     }
 
-    if (pointDimensionCount < 0) {
+    if (docValuesType == null) {
+      throw new IllegalStateException("DocValuesType must not be null (field: '" + name + "')");
+    }
+    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
       throw new IllegalStateException(
-          "pointDimensionCount must be >= 0; got " + pointDimensionCount);
+          "field '"
+              + name
+              + "' cannot have a docvalues update generation without having docvalues");
     }
 
+    if (pointDimensionCount < 0) {
+      throw new IllegalStateException(
+          "pointDimensionCount must be >= 0; got "
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
+    }
     if (pointIndexDimensionCount < 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be >= 0; got " + pointIndexDimensionCount);
+          "pointIndexDimensionCount must be >= 0; got "
+              + pointIndexDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes < 0) {
-      throw new IllegalStateException("pointNumBytes must be >= 0; got " + pointNumBytes);
+      throw new IllegalStateException(
+          "pointNumBytes must be >= 0; got " + pointNumBytes + " (field: '" + name + "')");
     }
 
     if (pointDimensionCount != 0 && pointNumBytes == 0) {
       throw new IllegalStateException(
-          "pointNumBytes must be > 0 when pointDimensionCount=" + pointDimensionCount);
+          "pointNumBytes must be > 0 when pointDimensionCount="
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointIndexDimensionCount != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be 0 when pointDimensionCount=0");
+          "pointIndexDimensionCount must be 0 when pointDimensionCount=0"
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointDimensionCount must be > 0 when pointNumBytes=" + pointNumBytes);
+          "pointDimensionCount must be > 0 when pointNumBytes="
+              + pointNumBytes
+              + " (field: '"
+              + name
+              + "')");
     }
 
-    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
+    if (vectorSearchStrategy == null) {
       throw new IllegalStateException(
-          "field '"
-              + name
-              + "' cannot have a docvalues update generation without having docvalues");
+          "Vector search strategy must not be null (field: '" + name + "')");
     }
-
     if (vectorDimension < 0) {
-      throw new IllegalStateException("vectorDimension must be >=0; got " + vectorDimension);
+      throw new IllegalStateException(
+          "vectorDimension must be >=0; got " + vectorDimension + " (field: '" + name + "')");
     }
-
     if (vectorDimension == 0 && vectorSearchStrategy != VectorValues.SearchStrategy.NONE) {
       throw new IllegalStateException(
-          "vector search strategy must be NONE when dimension = 0; got " + vectorSearchStrategy);
+          "vector search strategy must be NONE when dimension = 0; got "
+              + vectorSearchStrategy
+              + " (field: '"
+              + name
+              + "')");
     }
-
     return true;
   }
 
-  // should only be called by FieldInfos#addOrUpdate
-  void update(
-      boolean storeTermVector,
-      boolean omitNorms,
-      boolean storePayloads,
-      IndexOptions indexOptions,
-      Map<String, String> attributes,
-      int dimensionCount,
-      int indexDimensionCount,
-      int dimensionNumBytes) {
-    if (indexOptions == null) {
-      throw new NullPointerException("IndexOptions must not be null (field: \"" + name + "\")");
-    }
-    // System.out.println("FI.update field=" + name + " indexed=" + indexed + " omitNorms=" +
-    // omitNorms + " this.omitNorms=" + this.omitNorms);
-    if (this.indexOptions != indexOptions) {
-      if (this.indexOptions == IndexOptions.NONE) {
-        this.indexOptions = indexOptions;
-      } else if (indexOptions != IndexOptions.NONE) {
-        throw new IllegalArgumentException(
-            "cannot change field \""
-                + name
-                + "\" from index options="
-                + this.indexOptions
-                + " to inconsistent index options="
-                + indexOptions);
-      }
+  void verifySameSchema(FieldInfo o, long dvGen) {
+    String fieldName = this.name;
+    verifySameIndexOptions(fieldName, this.indexOptions, o.getIndexOptions());
+    if (this.indexOptions != IndexOptions.NONE) {
+      verifySameOmitNorms(fieldName, this.omitNorms, o.omitNorms);
+      verifySameStoreTermVectors(fieldName, this.storeTermVector, o.storeTermVector);
+    }
+    verifySameDocValuesType(fieldName, this.docValuesType, o.docValuesType);
+    verifySameDVGen(fieldName, this.dvGen, dvGen);
+    verifySamePointsOptions(
+        fieldName,
+        this.pointDimensionCount,
+        this.pointIndexDimensionCount,
+        this.pointNumBytes,
+        o.pointDimensionCount,
+        o.pointIndexDimensionCount,
+        o.pointNumBytes);
+    verifySameVectorOptions(
+        fieldName,
+        this.vectorDimension,
+        this.vectorSearchStrategy,
+        o.vectorDimension,
+        o.vectorSearchStrategy);
+  }
+
+  /**
+   * Verify that the provided index options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameIndexOptions(
+      String fieldName, IndexOptions indexOptions1, IndexOptions indexOptions2) {
+    if (indexOptions1 != indexOptions2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from index options="
+              + indexOptions1
+              + " to inconsistent index options="
+              + indexOptions2);
     }
+  }
 
-    if (this.pointDimensionCount == 0 && dimensionCount != 0) {
-      this.pointDimensionCount = dimensionCount;
-      this.pointIndexDimensionCount = indexDimensionCount;
-      this.pointNumBytes = dimensionNumBytes;
-    } else if (dimensionCount != 0
-        && (this.pointDimensionCount != dimensionCount
-            || this.pointIndexDimensionCount != indexDimensionCount
-            || this.pointNumBytes != dimensionNumBytes)) {
+  /**
+   * Verify that the provided docValues type are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameDocValuesType(
+      String fieldName, DocValuesType docValuesType1, DocValuesType docValuesType2) {
+    if (docValuesType1 != docValuesType2) {
       throw new IllegalArgumentException(
           "cannot change field \""
-              + name
+              + fieldName
+              + "\" from doc values type="
+              + docValuesType1
+              + " to inconsistent doc values type="
+              + docValuesType2);
+    }
+  }
+
+  /**
+   * Verify that the provided doc values generations are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  // TODO: not sure if gen also must be the same
+  public static void verifySameDVGen(String fieldName, long docValuesGen1, long docValuesGen2) {
+    if (docValuesGen1 != docValuesGen2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from doc values generation="
+              + docValuesGen1
+              + " to inconsistent doc values generation="
+              + docValuesGen2);
+    }
+  }
+
+  /**
+   * Verify that the provided store term vectors options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameStoreTermVectors(
+      String fieldName, boolean storeTermVector1, boolean storeTermVector2) {
+    if (storeTermVector1 != storeTermVector2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from storeTermVector="
+              + storeTermVector1
+              + " to inconsistent storeTermVector="
+              + storeTermVector2);
+    }
+  }
+
+  /**
+   * Verify that the provided omitNorms are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameOmitNorms(String fieldName, boolean omitNorms1, boolean omitNorms2) {
+    if (omitNorms1 != omitNorms2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from omitNorms="
+              + omitNorms1
+              + " to inconsistent omitNorms="
+              + omitNorms2);
+    }
+  }
+
+  /**
+   * Verify that the provided points indexing options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySamePointsOptions(
+      String fieldName,
+      int pointDimensionCount1,
+      int indexDimensionCount1,
+      int numBytes1,
+      int pointDimensionCount2,
+      int indexDimensionCount2,
+      int numBytes2) {
+    if (pointDimensionCount1 != pointDimensionCount2
+        || indexDimensionCount1 != indexDimensionCount2
+        || numBytes1 != numBytes2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
               + "\" from points dimensionCount="
-              + this.pointDimensionCount
+              + pointDimensionCount1
               + ", indexDimensionCount="
-              + this.pointIndexDimensionCount
+              + indexDimensionCount1
               + ", numBytes="
-              + this.pointNumBytes
+              + numBytes1
               + " to inconsistent dimensionCount="
-              + dimensionCount
+              + pointDimensionCount2
               + ", indexDimensionCount="
-              + indexDimensionCount
+              + indexDimensionCount2
               + ", numBytes="
-              + dimensionNumBytes);
+              + numBytes2);
     }
+  }
 
-    // if updated field data is not for indexing, leave the updates out
-    if (this.indexOptions != IndexOptions.NONE) {
-      this.storeTermVector |= storeTermVector; // once vector, always vector
-      this.storePayloads |= storePayloads;
-
-      // Awkward: only drop norms if incoming update is indexed:
-      if (indexOptions != IndexOptions.NONE && this.omitNorms != omitNorms) {
-        this.omitNorms = true; // if one require omitNorms at least once, it remains off for life
-      }
-    }
-    if (this.indexOptions == IndexOptions.NONE
-        || this.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
-      // cannot store payloads if we don't store positions:
-      this.storePayloads = false;
-    }
-    if (attributes != null) {
-      this.attributes.putAll(attributes);
+  /**
+   * Verify that the provided vector indexing options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameVectorOptions(

Review comment:
       can be pkg private?

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -796,167 +565,97 @@ public void add(FieldInfos other) {
     /** Create a new field, or return existing one. */
     public FieldInfo getOrAdd(String name) {
       FieldInfo fi = fieldInfo(name);
-      if (fi == null) {
-        assert assertNotFinished();
-        // This field wasn't yet added to this in-RAM
-        // segment's FieldInfo, so now we get a global
-        // number for this field.  If the field was seen
-        // before then we'll get the same name and number,
-        // else we'll allocate a new one:
-        final boolean isSoftDeletesField = name.equals(globalFieldNumbers.softDeletesFieldName);
-        final int fieldNumber =
-            globalFieldNumbers.addOrGet(
-                name,
-                -1,
-                IndexOptions.NONE,
-                DocValuesType.NONE,
-                0,
-                0,
-                0,
-                0,
-                VectorValues.SearchStrategy.NONE,
-                isSoftDeletesField);
-        fi =
-            new FieldInfo(
-                name,
-                fieldNumber,
-                false,
-                false,
-                false,
-                IndexOptions.NONE,
-                DocValuesType.NONE,
-                -1,
-                new HashMap<>(),
-                0,
-                0,
-                0,
-                0,
-                VectorValues.SearchStrategy.NONE,
-                isSoftDeletesField);
-        assert !byName.containsKey(fi.name);
-        globalFieldNumbers.verifyConsistent(
-            Integer.valueOf(fi.number), fi.name, DocValuesType.NONE);
-        byName.put(fi.name, fi);
+      if (fi != null) {
+        return fi;
+      } else {
+        return addField(
+            name,
+            -1,
+            false,
+            false,
+            false,
+            IndexOptions.NONE,
+            DocValuesType.NONE,
+            -1,
+            new HashMap<>(),
+            0,
+            0,
+            0,
+            0,
+            VectorValues.SearchStrategy.NONE,
+            name.equals(globalFieldNumbers.softDeletesFieldName));
       }
+    }
 
-      return fi;
+    public FieldInfo add(FieldInfo fi) {
+      return add(fi, -1);
+    }
+
+    public FieldInfo add(FieldInfo fi, long dvGen) {
+      // IMPORTANT - reuse the field number if possible for consistent field numbers across segments
+      if (fi.getDocValuesType() == null) {
+        throw new NullPointerException("DocValuesType must not be null");
+      }
+      final FieldInfo curFi = fieldInfo(fi.name);
+      if (curFi == null) {
+        // original attributes is UnmodifiableMap
+        Map<String, String> attributes =
+            fi.attributes() == null ? null : new HashMap<>(fi.attributes());
+        return addField(
+            fi.name,
+            fi.number,
+            fi.hasVectors(),
+            fi.omitsNorms(),
+            fi.hasPayloads(),
+            fi.getIndexOptions(),
+            fi.getDocValuesType(),
+            dvGen,
+            attributes,
+            fi.getPointDimensionCount(),
+            fi.getPointIndexDimensionCount(),
+            fi.getPointNumBytes(),
+            fi.getVectorDimension(),
+            fi.getVectorSearchStrategy(),
+            fi.isSoftDeletesField());
+      } else {
+        curFi.verifySameSchema(fi, dvGen);

Review comment:
       now that we don't pass the dvGen maybe we can use equals to compare? It might be controversial since we can't take the dvGen in to account.

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -130,127 +167,252 @@ public boolean checkConsistency() {
       }
     }
 
-    if (pointDimensionCount < 0) {
+    if (docValuesType == null) {
+      throw new IllegalStateException("DocValuesType must not be null (field: '" + name + "')");
+    }
+    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
       throw new IllegalStateException(
-          "pointDimensionCount must be >= 0; got " + pointDimensionCount);
+          "field '"
+              + name
+              + "' cannot have a docvalues update generation without having docvalues");
     }
 
+    if (pointDimensionCount < 0) {
+      throw new IllegalStateException(
+          "pointDimensionCount must be >= 0; got "
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
+    }
     if (pointIndexDimensionCount < 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be >= 0; got " + pointIndexDimensionCount);
+          "pointIndexDimensionCount must be >= 0; got "
+              + pointIndexDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes < 0) {
-      throw new IllegalStateException("pointNumBytes must be >= 0; got " + pointNumBytes);
+      throw new IllegalStateException(
+          "pointNumBytes must be >= 0; got " + pointNumBytes + " (field: '" + name + "')");
     }
 
     if (pointDimensionCount != 0 && pointNumBytes == 0) {
       throw new IllegalStateException(
-          "pointNumBytes must be > 0 when pointDimensionCount=" + pointDimensionCount);
+          "pointNumBytes must be > 0 when pointDimensionCount="
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointIndexDimensionCount != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be 0 when pointDimensionCount=0");
+          "pointIndexDimensionCount must be 0 when pointDimensionCount=0"
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointDimensionCount must be > 0 when pointNumBytes=" + pointNumBytes);
+          "pointDimensionCount must be > 0 when pointNumBytes="
+              + pointNumBytes
+              + " (field: '"
+              + name
+              + "')");
     }
 
-    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
+    if (vectorSearchStrategy == null) {
       throw new IllegalStateException(
-          "field '"
-              + name
-              + "' cannot have a docvalues update generation without having docvalues");
+          "Vector search strategy must not be null (field: '" + name + "')");
     }
-
     if (vectorDimension < 0) {
-      throw new IllegalStateException("vectorDimension must be >=0; got " + vectorDimension);
+      throw new IllegalStateException(
+          "vectorDimension must be >=0; got " + vectorDimension + " (field: '" + name + "')");
     }
-
     if (vectorDimension == 0 && vectorSearchStrategy != VectorValues.SearchStrategy.NONE) {
       throw new IllegalStateException(
-          "vector search strategy must be NONE when dimension = 0; got " + vectorSearchStrategy);
+          "vector search strategy must be NONE when dimension = 0; got "
+              + vectorSearchStrategy
+              + " (field: '"
+              + name
+              + "')");
     }
-
     return true;
   }
 
-  // should only be called by FieldInfos#addOrUpdate
-  void update(
-      boolean storeTermVector,
-      boolean omitNorms,
-      boolean storePayloads,
-      IndexOptions indexOptions,
-      Map<String, String> attributes,
-      int dimensionCount,
-      int indexDimensionCount,
-      int dimensionNumBytes) {
-    if (indexOptions == null) {
-      throw new NullPointerException("IndexOptions must not be null (field: \"" + name + "\")");
-    }
-    // System.out.println("FI.update field=" + name + " indexed=" + indexed + " omitNorms=" +
-    // omitNorms + " this.omitNorms=" + this.omitNorms);
-    if (this.indexOptions != indexOptions) {
-      if (this.indexOptions == IndexOptions.NONE) {
-        this.indexOptions = indexOptions;
-      } else if (indexOptions != IndexOptions.NONE) {
-        throw new IllegalArgumentException(
-            "cannot change field \""
-                + name
-                + "\" from index options="
-                + this.indexOptions
-                + " to inconsistent index options="
-                + indexOptions);
-      }
+  void verifySameSchema(FieldInfo o, long dvGen) {
+    String fieldName = this.name;
+    verifySameIndexOptions(fieldName, this.indexOptions, o.getIndexOptions());
+    if (this.indexOptions != IndexOptions.NONE) {
+      verifySameOmitNorms(fieldName, this.omitNorms, o.omitNorms);
+      verifySameStoreTermVectors(fieldName, this.storeTermVector, o.storeTermVector);
+    }
+    verifySameDocValuesType(fieldName, this.docValuesType, o.docValuesType);
+    verifySameDVGen(fieldName, this.dvGen, dvGen);
+    verifySamePointsOptions(
+        fieldName,
+        this.pointDimensionCount,
+        this.pointIndexDimensionCount,
+        this.pointNumBytes,
+        o.pointDimensionCount,
+        o.pointIndexDimensionCount,
+        o.pointNumBytes);
+    verifySameVectorOptions(
+        fieldName,
+        this.vectorDimension,
+        this.vectorSearchStrategy,
+        o.vectorDimension,
+        o.vectorSearchStrategy);
+  }
+
+  /**
+   * Verify that the provided index options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameIndexOptions(
+      String fieldName, IndexOptions indexOptions1, IndexOptions indexOptions2) {
+    if (indexOptions1 != indexOptions2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from index options="
+              + indexOptions1
+              + " to inconsistent index options="
+              + indexOptions2);
     }
+  }
 
-    if (this.pointDimensionCount == 0 && dimensionCount != 0) {
-      this.pointDimensionCount = dimensionCount;
-      this.pointIndexDimensionCount = indexDimensionCount;
-      this.pointNumBytes = dimensionNumBytes;
-    } else if (dimensionCount != 0
-        && (this.pointDimensionCount != dimensionCount
-            || this.pointIndexDimensionCount != indexDimensionCount
-            || this.pointNumBytes != dimensionNumBytes)) {
+  /**
+   * Verify that the provided docValues type are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameDocValuesType(
+      String fieldName, DocValuesType docValuesType1, DocValuesType docValuesType2) {
+    if (docValuesType1 != docValuesType2) {
       throw new IllegalArgumentException(
           "cannot change field \""
-              + name
+              + fieldName
+              + "\" from doc values type="
+              + docValuesType1
+              + " to inconsistent doc values type="
+              + docValuesType2);
+    }
+  }
+
+  /**
+   * Verify that the provided doc values generations are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  // TODO: not sure if gen also must be the same
+  public static void verifySameDVGen(String fieldName, long docValuesGen1, long docValuesGen2) {
+    if (docValuesGen1 != docValuesGen2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from doc values generation="
+              + docValuesGen1
+              + " to inconsistent doc values generation="
+              + docValuesGen2);
+    }
+  }
+
+  /**
+   * Verify that the provided store term vectors options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameStoreTermVectors(
+      String fieldName, boolean storeTermVector1, boolean storeTermVector2) {
+    if (storeTermVector1 != storeTermVector2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from storeTermVector="
+              + storeTermVector1
+              + " to inconsistent storeTermVector="
+              + storeTermVector2);
+    }
+  }
+
+  /**
+   * Verify that the provided omitNorms are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameOmitNorms(String fieldName, boolean omitNorms1, boolean omitNorms2) {

Review comment:
       can be pkg private?

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -796,167 +565,97 @@ public void add(FieldInfos other) {
     /** Create a new field, or return existing one. */
     public FieldInfo getOrAdd(String name) {
       FieldInfo fi = fieldInfo(name);
-      if (fi == null) {
-        assert assertNotFinished();
-        // This field wasn't yet added to this in-RAM
-        // segment's FieldInfo, so now we get a global
-        // number for this field.  If the field was seen
-        // before then we'll get the same name and number,
-        // else we'll allocate a new one:
-        final boolean isSoftDeletesField = name.equals(globalFieldNumbers.softDeletesFieldName);
-        final int fieldNumber =
-            globalFieldNumbers.addOrGet(
-                name,
-                -1,
-                IndexOptions.NONE,
-                DocValuesType.NONE,
-                0,
-                0,
-                0,
-                0,
-                VectorValues.SearchStrategy.NONE,
-                isSoftDeletesField);
-        fi =
-            new FieldInfo(
-                name,
-                fieldNumber,
-                false,
-                false,
-                false,
-                IndexOptions.NONE,
-                DocValuesType.NONE,
-                -1,
-                new HashMap<>(),
-                0,
-                0,
-                0,
-                0,
-                VectorValues.SearchStrategy.NONE,
-                isSoftDeletesField);
-        assert !byName.containsKey(fi.name);
-        globalFieldNumbers.verifyConsistent(
-            Integer.valueOf(fi.number), fi.name, DocValuesType.NONE);
-        byName.put(fi.name, fi);
+      if (fi != null) {
+        return fi;
+      } else {
+        return addField(
+            name,
+            -1,
+            false,
+            false,
+            false,
+            IndexOptions.NONE,
+            DocValuesType.NONE,
+            -1,
+            new HashMap<>(),
+            0,
+            0,
+            0,
+            0,
+            VectorValues.SearchStrategy.NONE,
+            name.equals(globalFieldNumbers.softDeletesFieldName));
       }
+    }
 
-      return fi;
+    public FieldInfo add(FieldInfo fi) {
+      return add(fi, -1);
+    }
+
+    public FieldInfo add(FieldInfo fi, long dvGen) {
+      // IMPORTANT - reuse the field number if possible for consistent field numbers across segments
+      if (fi.getDocValuesType() == null) {
+        throw new NullPointerException("DocValuesType must not be null");
+      }
+      final FieldInfo curFi = fieldInfo(fi.name);
+      if (curFi == null) {
+        // original attributes is UnmodifiableMap
+        Map<String, String> attributes =
+            fi.attributes() == null ? null : new HashMap<>(fi.attributes());
+        return addField(
+            fi.name,
+            fi.number,
+            fi.hasVectors(),
+            fi.omitsNorms(),
+            fi.hasPayloads(),
+            fi.getIndexOptions(),
+            fi.getDocValuesType(),
+            dvGen,
+            attributes,
+            fi.getPointDimensionCount(),
+            fi.getPointIndexDimensionCount(),
+            fi.getPointNumBytes(),
+            fi.getVectorDimension(),
+            fi.getVectorSearchStrategy(),
+            fi.isSoftDeletesField());
+      } else {
+        curFi.verifySameSchema(fi, dvGen);
+        if (fi.attributes() != null) {
+          fi.attributes().forEach((k, v) -> curFi.putAttribute(k, v));
+        }
+        return curFi;
+      }
     }
 
-    private FieldInfo addOrUpdateInternal(
+    public FieldInfo add(

Review comment:
       maybe we can safe some lines and put some args on the same line?

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -972,6 +671,64 @@ private boolean assertNotFinished() {
       return true;
     }
 
+    private FieldInfo addField(
+        String name,
+        int preferredFieldNumber,
+        boolean storeTermVector,
+        boolean omitNorms,
+        boolean storePayloads,
+        IndexOptions indexOptions,
+        DocValuesType docValues,
+        long dvGen,
+        Map<String, String> attributes,
+        int dataDimensionCount,
+        int indexDimensionCount,
+        int dimensionNumBytes,
+        int vectorDimension,
+        VectorValues.SearchStrategy vectorSearchStrategy,
+        boolean isSoftDeletesField) {
+      // This field wasn't yet added to this in-RAM
+      // segment's FieldInfo, so now we get a global
+      // number for this field.  If the field was seen
+      // before then we'll get the same name and number,
+      // else we'll allocate a new one:
+      assert assertNotFinished();
+      final int fieldNumber =
+          globalFieldNumbers.addOrGet(
+              name,
+              preferredFieldNumber,
+              indexOptions,
+              storeTermVector,
+              omitNorms,
+              docValues,
+              dataDimensionCount,
+              indexDimensionCount,
+              dimensionNumBytes,
+              vectorDimension,
+              vectorSearchStrategy,
+              isSoftDeletesField);
+      FieldInfo fi =
+          new FieldInfo(
+              name,
+              fieldNumber,
+              storeTermVector,
+              omitNorms,
+              storePayloads,
+              indexOptions,
+              docValues,
+              dvGen,
+              attributes,
+              dataDimensionCount,
+              indexDimensionCount,
+              dimensionNumBytes,
+              vectorDimension,
+              vectorSearchStrategy,
+              isSoftDeletesField);
+      assert byName.containsKey(fi.name) == false;

Review comment:
       can we have an assertion message here just in case

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -787,7 +557,6 @@ synchronized void setVectorDimensionsAndSearchStrategy(
     }
 
     public void add(FieldInfos other) {

Review comment:
       I think we can make this method pkg private too? 

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexingChain.java
##########
@@ -1313,4 +1307,118 @@ public void recycleIntBlocks(int[][] blocks, int offset, int length) {
       bytesUsed.addAndGet(-(length * (IntBlockPool.INT_BLOCK_SIZE * Integer.BYTES)));
     }
   }
+
+  /**
+   * A schema of the field in the current document. With every new document this schema is reset. As
+   * the document fields are processed, we update the schema with options encountered in this
+   * document. Once the processing for the document is done, we compare the built schema of the
+   * current document with the corresponding FieldInfo (FieldInfo is built on a first document in
+   * the segment where we encounter this field). If there is inconsistency, we raise an error. This
+   * ensures that a field has the same data structures across all documents.
+   */
+  private static final class FieldSchema {

Review comment:
       can't we fold this somehow into `FieldInfo` and then just create a single FieldInfo that we carry through out the indexing process for a segment. Maybe I am missing something which is absolutely possible but if we see FieldInfos / FieldNumbers as the source of truth it should never every change after we added it the first time or are there some parts of it that can change? I think it should be consistent for a segment at least except of attributes but they can be mutable for the segment? Maybe I am asking for too much, I really try to prevent having all these args again

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -972,6 +671,64 @@ private boolean assertNotFinished() {
       return true;
     }
 
+    private FieldInfo addField(

Review comment:
       we can also remove the dv gen from here I think

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -972,6 +671,64 @@ private boolean assertNotFinished() {
       return true;
     }
 
+    private FieldInfo addField(
+        String name,
+        int preferredFieldNumber,
+        boolean storeTermVector,
+        boolean omitNorms,
+        boolean storePayloads,
+        IndexOptions indexOptions,
+        DocValuesType docValues,
+        long dvGen,
+        Map<String, String> attributes,
+        int dataDimensionCount,
+        int indexDimensionCount,
+        int dimensionNumBytes,
+        int vectorDimension,
+        VectorValues.SearchStrategy vectorSearchStrategy,
+        boolean isSoftDeletesField) {
+      // This field wasn't yet added to this in-RAM
+      // segment's FieldInfo, so now we get a global
+      // number for this field.  If the field was seen
+      // before then we'll get the same name and number,
+      // else we'll allocate a new one:
+      assert assertNotFinished();
+      final int fieldNumber =
+          globalFieldNumbers.addOrGet(

Review comment:
       maybe we can simplify this a bit and return a `FieldInfo` from this call then we don't need to pass all the args again and we guarantee that we constructed what we checked? 

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -111,6 +111,43 @@ public FieldInfo(
 
   /** Performs internal consistency checks. Always returns true (or throws IllegalStateException) */
   public boolean checkConsistency() {
+    return checkOptionsCorrectness(
+        name,
+        storeTermVector,
+        omitNorms,
+        storePayloads,
+        indexOptions,
+        docValuesType,
+        dvGen,
+        pointDimensionCount,
+        pointIndexDimensionCount,
+        pointNumBytes,
+        vectorDimension,
+        vectorSearchStrategy);
+  }
+
+  /**
+   * Check correctness of FieldInfo options
+   *
+   * @throws IllegalStateException if some options are incorrect
+   * @return {@code true} if all options are correct
+   */
+  public static boolean checkOptionsCorrectness(
+      String name,
+      boolean storeTermVector,
+      boolean omitNorms,
+      boolean storePayloads,
+      IndexOptions indexOptions,
+      DocValuesType docValuesType,
+      long dvGen,
+      int pointDimensionCount,
+      int pointIndexDimensionCount,
+      int pointNumBytes,
+      int vectorDimension,
+      VectorValues.SearchStrategy vectorSearchStrategy) {
+    if (indexOptions == null) {
+      throw new IllegalStateException("IndexOptions must not be null (field: '" + name + "')");

Review comment:
       since we are including the field name in every exception now maybe it might make sense to have a static method with _String field, String msg_ to not repeat yourself all other the place?

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexingChain.java
##########
@@ -581,78 +584,160 @@ private void finishStoredFields() throws IOException {
   }
 
   void processDocument(int docID, Iterable<? extends IndexableField> document) throws IOException {
-
-    // How many indexed field names we've seen (collapses
-    // multiple field instances by the same name):
-    int fieldCount = 0;
-
+    int fieldCount =
+        0; // How many field names we've seen (collapses multiple field instances by the same name)
+    int indexedFieldCount = 0; // number of unique fields indexed with postings
     long fieldGen = nextFieldGen++;
-
-    // NOTE: we need two passes here, in case there are
-    // multi-valued fields, because we must process all
-    // instances of a given field at once, since the
-    // analyzer is free to reuse TokenStream across fields
-    // (i.e., we cannot have more than one TokenStream
-    // running "at once"):
+    int docFieldIdx = 0;
 
     termsHash.startDocument();
-
     startStoredFields(docID);
     try {
+      // 1st pass over doc fields – verify that doc schema matches the index schema
+      // build schema for each unique doc field
+      for (IndexableField field : document) {
+        IndexableFieldType fieldType = field.fieldType();
+        PerField pf = getOrAddPerField(field.name(), fieldType);
+        if (pf.fieldGen != fieldGen) { // first time we see this field in this document
+          fields[fieldCount++] = pf;
+          pf.fieldGen = fieldGen;
+          pf.reset(docID);
+        }
+        if (docFieldIdx >= docFields.length) oversizeDocFields();
+        docFields[docFieldIdx++] = pf;
+        updateDocFieldSchema(field.name(), pf.schema, fieldType);
+      }
+      // for each field verify that its schema within the current doc matches its schema in the
+      // index
+      for (int i = 0; i < fieldCount; i++) {
+        PerField pf = fields[i];
+        if (pf.fieldInfo == null) { // the first time we see this field in this segment
+          initializeFieldInfo(pf);
+        } else {
+          pf.schema.assertSameSchema(pf.fieldInfo);
+        }
+      }
+
+      // 2nd pass over doc fields – index each field
+      // also count the number of unique fields indexed with postings
+      docFieldIdx = 0;
       for (IndexableField field : document) {
-        fieldCount = processField(docID, field, fieldGen, fieldCount);
+        if (processField(docID, field, docFields[docFieldIdx])) {
+          fields[indexedFieldCount] = docFields[docFieldIdx];
+          indexedFieldCount++;
+        }
+        docFieldIdx++;
       }
     } finally {
       if (hasHitAbortingException == false) {
         // Finish each indexed field name seen in the document:
-        for (int i = 0; i < fieldCount; i++) {
+        for (int i = 0; i < indexedFieldCount; i++) {
           fields[i].finish(docID);
         }
         finishStoredFields();
+        // TODO: for broken docs, optimize termsHash.finishDocument

Review comment:
       did you add this? What is there to optimize? :)

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -130,127 +167,252 @@ public boolean checkConsistency() {
       }
     }
 
-    if (pointDimensionCount < 0) {
+    if (docValuesType == null) {
+      throw new IllegalStateException("DocValuesType must not be null (field: '" + name + "')");
+    }
+    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
       throw new IllegalStateException(
-          "pointDimensionCount must be >= 0; got " + pointDimensionCount);
+          "field '"
+              + name
+              + "' cannot have a docvalues update generation without having docvalues");
     }
 
+    if (pointDimensionCount < 0) {
+      throw new IllegalStateException(
+          "pointDimensionCount must be >= 0; got "
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
+    }
     if (pointIndexDimensionCount < 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be >= 0; got " + pointIndexDimensionCount);
+          "pointIndexDimensionCount must be >= 0; got "
+              + pointIndexDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes < 0) {
-      throw new IllegalStateException("pointNumBytes must be >= 0; got " + pointNumBytes);
+      throw new IllegalStateException(
+          "pointNumBytes must be >= 0; got " + pointNumBytes + " (field: '" + name + "')");
     }
 
     if (pointDimensionCount != 0 && pointNumBytes == 0) {
       throw new IllegalStateException(
-          "pointNumBytes must be > 0 when pointDimensionCount=" + pointDimensionCount);
+          "pointNumBytes must be > 0 when pointDimensionCount="
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointIndexDimensionCount != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be 0 when pointDimensionCount=0");
+          "pointIndexDimensionCount must be 0 when pointDimensionCount=0"
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointDimensionCount must be > 0 when pointNumBytes=" + pointNumBytes);
+          "pointDimensionCount must be > 0 when pointNumBytes="
+              + pointNumBytes
+              + " (field: '"
+              + name
+              + "')");
     }
 
-    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
+    if (vectorSearchStrategy == null) {
       throw new IllegalStateException(
-          "field '"
-              + name
-              + "' cannot have a docvalues update generation without having docvalues");
+          "Vector search strategy must not be null (field: '" + name + "')");
     }
-
     if (vectorDimension < 0) {
-      throw new IllegalStateException("vectorDimension must be >=0; got " + vectorDimension);
+      throw new IllegalStateException(
+          "vectorDimension must be >=0; got " + vectorDimension + " (field: '" + name + "')");
     }
-
     if (vectorDimension == 0 && vectorSearchStrategy != VectorValues.SearchStrategy.NONE) {
       throw new IllegalStateException(
-          "vector search strategy must be NONE when dimension = 0; got " + vectorSearchStrategy);
+          "vector search strategy must be NONE when dimension = 0; got "
+              + vectorSearchStrategy
+              + " (field: '"
+              + name
+              + "')");
     }
-
     return true;
   }
 
-  // should only be called by FieldInfos#addOrUpdate
-  void update(
-      boolean storeTermVector,
-      boolean omitNorms,
-      boolean storePayloads,
-      IndexOptions indexOptions,
-      Map<String, String> attributes,
-      int dimensionCount,
-      int indexDimensionCount,
-      int dimensionNumBytes) {
-    if (indexOptions == null) {
-      throw new NullPointerException("IndexOptions must not be null (field: \"" + name + "\")");
-    }
-    // System.out.println("FI.update field=" + name + " indexed=" + indexed + " omitNorms=" +
-    // omitNorms + " this.omitNorms=" + this.omitNorms);
-    if (this.indexOptions != indexOptions) {
-      if (this.indexOptions == IndexOptions.NONE) {
-        this.indexOptions = indexOptions;
-      } else if (indexOptions != IndexOptions.NONE) {
-        throw new IllegalArgumentException(
-            "cannot change field \""
-                + name
-                + "\" from index options="
-                + this.indexOptions
-                + " to inconsistent index options="
-                + indexOptions);
-      }
+  void verifySameSchema(FieldInfo o, long dvGen) {
+    String fieldName = this.name;
+    verifySameIndexOptions(fieldName, this.indexOptions, o.getIndexOptions());
+    if (this.indexOptions != IndexOptions.NONE) {
+      verifySameOmitNorms(fieldName, this.omitNorms, o.omitNorms);
+      verifySameStoreTermVectors(fieldName, this.storeTermVector, o.storeTermVector);
+    }
+    verifySameDocValuesType(fieldName, this.docValuesType, o.docValuesType);
+    verifySameDVGen(fieldName, this.dvGen, dvGen);
+    verifySamePointsOptions(
+        fieldName,
+        this.pointDimensionCount,
+        this.pointIndexDimensionCount,
+        this.pointNumBytes,
+        o.pointDimensionCount,
+        o.pointIndexDimensionCount,
+        o.pointNumBytes);
+    verifySameVectorOptions(
+        fieldName,
+        this.vectorDimension,
+        this.vectorSearchStrategy,
+        o.vectorDimension,
+        o.vectorSearchStrategy);
+  }
+
+  /**
+   * Verify that the provided index options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameIndexOptions(
+      String fieldName, IndexOptions indexOptions1, IndexOptions indexOptions2) {
+    if (indexOptions1 != indexOptions2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from index options="
+              + indexOptions1
+              + " to inconsistent index options="
+              + indexOptions2);
     }
+  }
 
-    if (this.pointDimensionCount == 0 && dimensionCount != 0) {
-      this.pointDimensionCount = dimensionCount;
-      this.pointIndexDimensionCount = indexDimensionCount;
-      this.pointNumBytes = dimensionNumBytes;
-    } else if (dimensionCount != 0
-        && (this.pointDimensionCount != dimensionCount
-            || this.pointIndexDimensionCount != indexDimensionCount
-            || this.pointNumBytes != dimensionNumBytes)) {
+  /**
+   * Verify that the provided docValues type are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameDocValuesType(
+      String fieldName, DocValuesType docValuesType1, DocValuesType docValuesType2) {
+    if (docValuesType1 != docValuesType2) {
       throw new IllegalArgumentException(
           "cannot change field \""
-              + name
+              + fieldName
+              + "\" from doc values type="
+              + docValuesType1
+              + " to inconsistent doc values type="
+              + docValuesType2);
+    }
+  }
+
+  /**
+   * Verify that the provided doc values generations are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  // TODO: not sure if gen also must be the same
+  public static void verifySameDVGen(String fieldName, long docValuesGen1, long docValuesGen2) {

Review comment:
       can this be a private method?

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -787,7 +557,6 @@ synchronized void setVectorDimensionsAndSearchStrategy(
     }
 
     public void add(FieldInfos other) {

Review comment:
       or even merge it into `getMergedFieldInfos`

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -130,127 +167,252 @@ public boolean checkConsistency() {
       }
     }
 
-    if (pointDimensionCount < 0) {
+    if (docValuesType == null) {
+      throw new IllegalStateException("DocValuesType must not be null (field: '" + name + "')");
+    }
+    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
       throw new IllegalStateException(
-          "pointDimensionCount must be >= 0; got " + pointDimensionCount);
+          "field '"
+              + name
+              + "' cannot have a docvalues update generation without having docvalues");
     }
 
+    if (pointDimensionCount < 0) {
+      throw new IllegalStateException(
+          "pointDimensionCount must be >= 0; got "
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
+    }
     if (pointIndexDimensionCount < 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be >= 0; got " + pointIndexDimensionCount);
+          "pointIndexDimensionCount must be >= 0; got "
+              + pointIndexDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes < 0) {
-      throw new IllegalStateException("pointNumBytes must be >= 0; got " + pointNumBytes);
+      throw new IllegalStateException(
+          "pointNumBytes must be >= 0; got " + pointNumBytes + " (field: '" + name + "')");
     }
 
     if (pointDimensionCount != 0 && pointNumBytes == 0) {
       throw new IllegalStateException(
-          "pointNumBytes must be > 0 when pointDimensionCount=" + pointDimensionCount);
+          "pointNumBytes must be > 0 when pointDimensionCount="
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointIndexDimensionCount != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be 0 when pointDimensionCount=0");
+          "pointIndexDimensionCount must be 0 when pointDimensionCount=0"
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointDimensionCount must be > 0 when pointNumBytes=" + pointNumBytes);
+          "pointDimensionCount must be > 0 when pointNumBytes="
+              + pointNumBytes
+              + " (field: '"
+              + name
+              + "')");
     }
 
-    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
+    if (vectorSearchStrategy == null) {
       throw new IllegalStateException(
-          "field '"
-              + name
-              + "' cannot have a docvalues update generation without having docvalues");
+          "Vector search strategy must not be null (field: '" + name + "')");
     }
-
     if (vectorDimension < 0) {
-      throw new IllegalStateException("vectorDimension must be >=0; got " + vectorDimension);
+      throw new IllegalStateException(
+          "vectorDimension must be >=0; got " + vectorDimension + " (field: '" + name + "')");
     }
-
     if (vectorDimension == 0 && vectorSearchStrategy != VectorValues.SearchStrategy.NONE) {
       throw new IllegalStateException(
-          "vector search strategy must be NONE when dimension = 0; got " + vectorSearchStrategy);
+          "vector search strategy must be NONE when dimension = 0; got "
+              + vectorSearchStrategy
+              + " (field: '"
+              + name
+              + "')");
     }
-
     return true;
   }
 
-  // should only be called by FieldInfos#addOrUpdate
-  void update(
-      boolean storeTermVector,
-      boolean omitNorms,
-      boolean storePayloads,
-      IndexOptions indexOptions,
-      Map<String, String> attributes,
-      int dimensionCount,
-      int indexDimensionCount,
-      int dimensionNumBytes) {
-    if (indexOptions == null) {
-      throw new NullPointerException("IndexOptions must not be null (field: \"" + name + "\")");
-    }
-    // System.out.println("FI.update field=" + name + " indexed=" + indexed + " omitNorms=" +
-    // omitNorms + " this.omitNorms=" + this.omitNorms);
-    if (this.indexOptions != indexOptions) {
-      if (this.indexOptions == IndexOptions.NONE) {
-        this.indexOptions = indexOptions;
-      } else if (indexOptions != IndexOptions.NONE) {
-        throw new IllegalArgumentException(
-            "cannot change field \""
-                + name
-                + "\" from index options="
-                + this.indexOptions
-                + " to inconsistent index options="
-                + indexOptions);
-      }
+  void verifySameSchema(FieldInfo o, long dvGen) {
+    String fieldName = this.name;
+    verifySameIndexOptions(fieldName, this.indexOptions, o.getIndexOptions());
+    if (this.indexOptions != IndexOptions.NONE) {
+      verifySameOmitNorms(fieldName, this.omitNorms, o.omitNorms);
+      verifySameStoreTermVectors(fieldName, this.storeTermVector, o.storeTermVector);
+    }
+    verifySameDocValuesType(fieldName, this.docValuesType, o.docValuesType);
+    verifySameDVGen(fieldName, this.dvGen, dvGen);
+    verifySamePointsOptions(
+        fieldName,
+        this.pointDimensionCount,
+        this.pointIndexDimensionCount,
+        this.pointNumBytes,
+        o.pointDimensionCount,
+        o.pointIndexDimensionCount,
+        o.pointNumBytes);
+    verifySameVectorOptions(
+        fieldName,
+        this.vectorDimension,
+        this.vectorSearchStrategy,
+        o.vectorDimension,
+        o.vectorSearchStrategy);
+  }
+
+  /**
+   * Verify that the provided index options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameIndexOptions(
+      String fieldName, IndexOptions indexOptions1, IndexOptions indexOptions2) {
+    if (indexOptions1 != indexOptions2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from index options="
+              + indexOptions1
+              + " to inconsistent index options="
+              + indexOptions2);
     }
+  }
 
-    if (this.pointDimensionCount == 0 && dimensionCount != 0) {
-      this.pointDimensionCount = dimensionCount;
-      this.pointIndexDimensionCount = indexDimensionCount;
-      this.pointNumBytes = dimensionNumBytes;
-    } else if (dimensionCount != 0
-        && (this.pointDimensionCount != dimensionCount
-            || this.pointIndexDimensionCount != indexDimensionCount
-            || this.pointNumBytes != dimensionNumBytes)) {
+  /**
+   * Verify that the provided docValues type are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameDocValuesType(
+      String fieldName, DocValuesType docValuesType1, DocValuesType docValuesType2) {
+    if (docValuesType1 != docValuesType2) {
       throw new IllegalArgumentException(
           "cannot change field \""
-              + name
+              + fieldName
+              + "\" from doc values type="
+              + docValuesType1
+              + " to inconsistent doc values type="
+              + docValuesType2);
+    }
+  }
+
+  /**
+   * Verify that the provided doc values generations are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  // TODO: not sure if gen also must be the same
+  public static void verifySameDVGen(String fieldName, long docValuesGen1, long docValuesGen2) {

Review comment:
       in fact I think we can remove it, see my comment here https://github.com/apache/lucene/pull/11/files?diff=unified&w=1#diff-b81932cf39b70347481b9659dd55b9b373427c2992cdacdfbc10ff8f4ea9eec0R594

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -796,167 +565,97 @@ public void add(FieldInfos other) {
     /** Create a new field, or return existing one. */
     public FieldInfo getOrAdd(String name) {
       FieldInfo fi = fieldInfo(name);
-      if (fi == null) {
-        assert assertNotFinished();
-        // This field wasn't yet added to this in-RAM
-        // segment's FieldInfo, so now we get a global
-        // number for this field.  If the field was seen
-        // before then we'll get the same name and number,
-        // else we'll allocate a new one:
-        final boolean isSoftDeletesField = name.equals(globalFieldNumbers.softDeletesFieldName);
-        final int fieldNumber =
-            globalFieldNumbers.addOrGet(
-                name,
-                -1,
-                IndexOptions.NONE,
-                DocValuesType.NONE,
-                0,
-                0,
-                0,
-                0,
-                VectorValues.SearchStrategy.NONE,
-                isSoftDeletesField);
-        fi =
-            new FieldInfo(
-                name,
-                fieldNumber,
-                false,
-                false,
-                false,
-                IndexOptions.NONE,
-                DocValuesType.NONE,
-                -1,
-                new HashMap<>(),
-                0,
-                0,
-                0,
-                0,
-                VectorValues.SearchStrategy.NONE,
-                isSoftDeletesField);
-        assert !byName.containsKey(fi.name);
-        globalFieldNumbers.verifyConsistent(
-            Integer.valueOf(fi.number), fi.name, DocValuesType.NONE);
-        byName.put(fi.name, fi);
+      if (fi != null) {
+        return fi;
+      } else {
+        return addField(
+            name,
+            -1,
+            false,
+            false,
+            false,
+            IndexOptions.NONE,
+            DocValuesType.NONE,
+            -1,
+            new HashMap<>(),
+            0,
+            0,
+            0,
+            0,
+            VectorValues.SearchStrategy.NONE,
+            name.equals(globalFieldNumbers.softDeletesFieldName));
       }
+    }
 
-      return fi;
+    public FieldInfo add(FieldInfo fi) {
+      return add(fi, -1);
+    }
+
+    public FieldInfo add(FieldInfo fi, long dvGen) {

Review comment:
       I think we can remove this method and only expose the `add(FieldInfo fi)` the DV gen should be irrelevant in all the cases when this is called (here in the builder and in the ParallelLeafReader) In fact, I think it would actually be wrong to enforce they are consistent. There might be 2 parallel leaf readers that have different DV gens and it will fail while that's ok?! I think we don't have a test for this at all. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on pull request #11:
URL: https://github.com/apache/lucene/pull/11#issuecomment-807089871


   @rmuir Thank you for checking and your feedback.
   
   > Could we arrange the tests better to make it easy to write lots of simpler tests?
   
   I've tried to address your feedback in 1246e402ea51af1b8f6c7cad54a1023488daa786. But I am not sure, if it is a good enough arrangment. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r598288722



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -787,7 +557,6 @@ synchronized void setVectorDimensionsAndSearchStrategy(
     }
 
     public void add(FieldInfos other) {

Review comment:
       Addressed in 6dc9f27 and 55fa8c8




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r596388076



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -130,127 +167,252 @@ public boolean checkConsistency() {
       }
     }
 
-    if (pointDimensionCount < 0) {
+    if (docValuesType == null) {
+      throw new IllegalStateException("DocValuesType must not be null (field: '" + name + "')");
+    }
+    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
       throw new IllegalStateException(
-          "pointDimensionCount must be >= 0; got " + pointDimensionCount);
+          "field '"
+              + name
+              + "' cannot have a docvalues update generation without having docvalues");
     }
 
+    if (pointDimensionCount < 0) {
+      throw new IllegalStateException(
+          "pointDimensionCount must be >= 0; got "
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
+    }
     if (pointIndexDimensionCount < 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be >= 0; got " + pointIndexDimensionCount);
+          "pointIndexDimensionCount must be >= 0; got "
+              + pointIndexDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes < 0) {
-      throw new IllegalStateException("pointNumBytes must be >= 0; got " + pointNumBytes);
+      throw new IllegalStateException(
+          "pointNumBytes must be >= 0; got " + pointNumBytes + " (field: '" + name + "')");
     }
 
     if (pointDimensionCount != 0 && pointNumBytes == 0) {
       throw new IllegalStateException(
-          "pointNumBytes must be > 0 when pointDimensionCount=" + pointDimensionCount);
+          "pointNumBytes must be > 0 when pointDimensionCount="
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointIndexDimensionCount != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be 0 when pointDimensionCount=0");
+          "pointIndexDimensionCount must be 0 when pointDimensionCount=0"
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointDimensionCount must be > 0 when pointNumBytes=" + pointNumBytes);
+          "pointDimensionCount must be > 0 when pointNumBytes="
+              + pointNumBytes
+              + " (field: '"
+              + name
+              + "')");
     }
 
-    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
+    if (vectorSearchStrategy == null) {
       throw new IllegalStateException(
-          "field '"
-              + name
-              + "' cannot have a docvalues update generation without having docvalues");
+          "Vector search strategy must not be null (field: '" + name + "')");
     }
-
     if (vectorDimension < 0) {
-      throw new IllegalStateException("vectorDimension must be >=0; got " + vectorDimension);
+      throw new IllegalStateException(
+          "vectorDimension must be >=0; got " + vectorDimension + " (field: '" + name + "')");
     }
-
     if (vectorDimension == 0 && vectorSearchStrategy != VectorValues.SearchStrategy.NONE) {
       throw new IllegalStateException(
-          "vector search strategy must be NONE when dimension = 0; got " + vectorSearchStrategy);
+          "vector search strategy must be NONE when dimension = 0; got "
+              + vectorSearchStrategy
+              + " (field: '"
+              + name
+              + "')");
     }
-
     return true;
   }
 
-  // should only be called by FieldInfos#addOrUpdate
-  void update(
-      boolean storeTermVector,
-      boolean omitNorms,
-      boolean storePayloads,
-      IndexOptions indexOptions,
-      Map<String, String> attributes,
-      int dimensionCount,
-      int indexDimensionCount,
-      int dimensionNumBytes) {
-    if (indexOptions == null) {
-      throw new NullPointerException("IndexOptions must not be null (field: \"" + name + "\")");
-    }
-    // System.out.println("FI.update field=" + name + " indexed=" + indexed + " omitNorms=" +
-    // omitNorms + " this.omitNorms=" + this.omitNorms);
-    if (this.indexOptions != indexOptions) {
-      if (this.indexOptions == IndexOptions.NONE) {
-        this.indexOptions = indexOptions;
-      } else if (indexOptions != IndexOptions.NONE) {
-        throw new IllegalArgumentException(
-            "cannot change field \""
-                + name
-                + "\" from index options="
-                + this.indexOptions
-                + " to inconsistent index options="
-                + indexOptions);
-      }
+  void verifySameSchema(FieldInfo o, long dvGen) {
+    String fieldName = this.name;
+    verifySameIndexOptions(fieldName, this.indexOptions, o.getIndexOptions());
+    if (this.indexOptions != IndexOptions.NONE) {
+      verifySameOmitNorms(fieldName, this.omitNorms, o.omitNorms);
+      verifySameStoreTermVectors(fieldName, this.storeTermVector, o.storeTermVector);
+    }
+    verifySameDocValuesType(fieldName, this.docValuesType, o.docValuesType);
+    verifySameDVGen(fieldName, this.dvGen, dvGen);
+    verifySamePointsOptions(
+        fieldName,
+        this.pointDimensionCount,
+        this.pointIndexDimensionCount,
+        this.pointNumBytes,
+        o.pointDimensionCount,
+        o.pointIndexDimensionCount,
+        o.pointNumBytes);
+    verifySameVectorOptions(
+        fieldName,
+        this.vectorDimension,
+        this.vectorSearchStrategy,
+        o.vectorDimension,
+        o.vectorSearchStrategy);
+  }
+
+  /**
+   * Verify that the provided index options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameIndexOptions(
+      String fieldName, IndexOptions indexOptions1, IndexOptions indexOptions2) {
+    if (indexOptions1 != indexOptions2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from index options="
+              + indexOptions1
+              + " to inconsistent index options="
+              + indexOptions2);
     }
+  }
 
-    if (this.pointDimensionCount == 0 && dimensionCount != 0) {
-      this.pointDimensionCount = dimensionCount;
-      this.pointIndexDimensionCount = indexDimensionCount;
-      this.pointNumBytes = dimensionNumBytes;
-    } else if (dimensionCount != 0
-        && (this.pointDimensionCount != dimensionCount
-            || this.pointIndexDimensionCount != indexDimensionCount
-            || this.pointNumBytes != dimensionNumBytes)) {
+  /**
+   * Verify that the provided docValues type are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameDocValuesType(
+      String fieldName, DocValuesType docValuesType1, DocValuesType docValuesType2) {
+    if (docValuesType1 != docValuesType2) {
       throw new IllegalArgumentException(
           "cannot change field \""
-              + name
+              + fieldName
+              + "\" from doc values type="
+              + docValuesType1
+              + " to inconsistent doc values type="
+              + docValuesType2);
+    }
+  }
+
+  /**
+   * Verify that the provided doc values generations are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  // TODO: not sure if gen also must be the same
+  public static void verifySameDVGen(String fieldName, long docValuesGen1, long docValuesGen2) {
+    if (docValuesGen1 != docValuesGen2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from doc values generation="
+              + docValuesGen1
+              + " to inconsistent doc values generation="
+              + docValuesGen2);
+    }
+  }
+
+  /**
+   * Verify that the provided store term vectors options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameStoreTermVectors(
+      String fieldName, boolean storeTermVector1, boolean storeTermVector2) {
+    if (storeTermVector1 != storeTermVector2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from storeTermVector="
+              + storeTermVector1
+              + " to inconsistent storeTermVector="
+              + storeTermVector2);
+    }
+  }
+
+  /**
+   * Verify that the provided omitNorms are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameOmitNorms(String fieldName, boolean omitNorms1, boolean omitNorms2) {

Review comment:
       Addressed in 0fe3493110ac2a5f750ad41f732436daff6c69f5




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #11: LUCENE-9334 Consistency of field data structures

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #11:
URL: https://github.com/apache/lucene/pull/11#discussion_r596387988



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -130,127 +167,252 @@ public boolean checkConsistency() {
       }
     }
 
-    if (pointDimensionCount < 0) {
+    if (docValuesType == null) {
+      throw new IllegalStateException("DocValuesType must not be null (field: '" + name + "')");
+    }
+    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
       throw new IllegalStateException(
-          "pointDimensionCount must be >= 0; got " + pointDimensionCount);
+          "field '"
+              + name
+              + "' cannot have a docvalues update generation without having docvalues");
     }
 
+    if (pointDimensionCount < 0) {
+      throw new IllegalStateException(
+          "pointDimensionCount must be >= 0; got "
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
+    }
     if (pointIndexDimensionCount < 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be >= 0; got " + pointIndexDimensionCount);
+          "pointIndexDimensionCount must be >= 0; got "
+              + pointIndexDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes < 0) {
-      throw new IllegalStateException("pointNumBytes must be >= 0; got " + pointNumBytes);
+      throw new IllegalStateException(
+          "pointNumBytes must be >= 0; got " + pointNumBytes + " (field: '" + name + "')");
     }
 
     if (pointDimensionCount != 0 && pointNumBytes == 0) {
       throw new IllegalStateException(
-          "pointNumBytes must be > 0 when pointDimensionCount=" + pointDimensionCount);
+          "pointNumBytes must be > 0 when pointDimensionCount="
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointIndexDimensionCount != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointIndexDimensionCount must be 0 when pointDimensionCount=0");
+          "pointIndexDimensionCount must be 0 when pointDimensionCount=0"
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes != 0 && pointDimensionCount == 0) {
       throw new IllegalStateException(
-          "pointDimensionCount must be > 0 when pointNumBytes=" + pointNumBytes);
+          "pointDimensionCount must be > 0 when pointNumBytes="
+              + pointNumBytes
+              + " (field: '"
+              + name
+              + "')");
     }
 
-    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
+    if (vectorSearchStrategy == null) {
       throw new IllegalStateException(
-          "field '"
-              + name
-              + "' cannot have a docvalues update generation without having docvalues");
+          "Vector search strategy must not be null (field: '" + name + "')");
     }
-
     if (vectorDimension < 0) {
-      throw new IllegalStateException("vectorDimension must be >=0; got " + vectorDimension);
+      throw new IllegalStateException(
+          "vectorDimension must be >=0; got " + vectorDimension + " (field: '" + name + "')");
     }
-
     if (vectorDimension == 0 && vectorSearchStrategy != VectorValues.SearchStrategy.NONE) {
       throw new IllegalStateException(
-          "vector search strategy must be NONE when dimension = 0; got " + vectorSearchStrategy);
+          "vector search strategy must be NONE when dimension = 0; got "
+              + vectorSearchStrategy
+              + " (field: '"
+              + name
+              + "')");
     }
-
     return true;
   }
 
-  // should only be called by FieldInfos#addOrUpdate
-  void update(
-      boolean storeTermVector,
-      boolean omitNorms,
-      boolean storePayloads,
-      IndexOptions indexOptions,
-      Map<String, String> attributes,
-      int dimensionCount,
-      int indexDimensionCount,
-      int dimensionNumBytes) {
-    if (indexOptions == null) {
-      throw new NullPointerException("IndexOptions must not be null (field: \"" + name + "\")");
-    }
-    // System.out.println("FI.update field=" + name + " indexed=" + indexed + " omitNorms=" +
-    // omitNorms + " this.omitNorms=" + this.omitNorms);
-    if (this.indexOptions != indexOptions) {
-      if (this.indexOptions == IndexOptions.NONE) {
-        this.indexOptions = indexOptions;
-      } else if (indexOptions != IndexOptions.NONE) {
-        throw new IllegalArgumentException(
-            "cannot change field \""
-                + name
-                + "\" from index options="
-                + this.indexOptions
-                + " to inconsistent index options="
-                + indexOptions);
-      }
+  void verifySameSchema(FieldInfo o, long dvGen) {
+    String fieldName = this.name;
+    verifySameIndexOptions(fieldName, this.indexOptions, o.getIndexOptions());
+    if (this.indexOptions != IndexOptions.NONE) {
+      verifySameOmitNorms(fieldName, this.omitNorms, o.omitNorms);
+      verifySameStoreTermVectors(fieldName, this.storeTermVector, o.storeTermVector);
+    }
+    verifySameDocValuesType(fieldName, this.docValuesType, o.docValuesType);
+    verifySameDVGen(fieldName, this.dvGen, dvGen);
+    verifySamePointsOptions(
+        fieldName,
+        this.pointDimensionCount,
+        this.pointIndexDimensionCount,
+        this.pointNumBytes,
+        o.pointDimensionCount,
+        o.pointIndexDimensionCount,
+        o.pointNumBytes);
+    verifySameVectorOptions(
+        fieldName,
+        this.vectorDimension,
+        this.vectorSearchStrategy,
+        o.vectorDimension,
+        o.vectorSearchStrategy);
+  }
+
+  /**
+   * Verify that the provided index options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameIndexOptions(
+      String fieldName, IndexOptions indexOptions1, IndexOptions indexOptions2) {
+    if (indexOptions1 != indexOptions2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from index options="
+              + indexOptions1
+              + " to inconsistent index options="
+              + indexOptions2);
     }
+  }
 
-    if (this.pointDimensionCount == 0 && dimensionCount != 0) {
-      this.pointDimensionCount = dimensionCount;
-      this.pointIndexDimensionCount = indexDimensionCount;
-      this.pointNumBytes = dimensionNumBytes;
-    } else if (dimensionCount != 0
-        && (this.pointDimensionCount != dimensionCount
-            || this.pointIndexDimensionCount != indexDimensionCount
-            || this.pointNumBytes != dimensionNumBytes)) {
+  /**
+   * Verify that the provided docValues type are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameDocValuesType(
+      String fieldName, DocValuesType docValuesType1, DocValuesType docValuesType2) {
+    if (docValuesType1 != docValuesType2) {
       throw new IllegalArgumentException(
           "cannot change field \""
-              + name
+              + fieldName
+              + "\" from doc values type="
+              + docValuesType1
+              + " to inconsistent doc values type="
+              + docValuesType2);
+    }
+  }
+
+  /**
+   * Verify that the provided doc values generations are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  // TODO: not sure if gen also must be the same
+  public static void verifySameDVGen(String fieldName, long docValuesGen1, long docValuesGen2) {
+    if (docValuesGen1 != docValuesGen2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from doc values generation="
+              + docValuesGen1
+              + " to inconsistent doc values generation="
+              + docValuesGen2);
+    }
+  }
+
+  /**
+   * Verify that the provided store term vectors options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameStoreTermVectors(
+      String fieldName, boolean storeTermVector1, boolean storeTermVector2) {
+    if (storeTermVector1 != storeTermVector2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from storeTermVector="
+              + storeTermVector1
+              + " to inconsistent storeTermVector="
+              + storeTermVector2);
+    }
+  }
+
+  /**
+   * Verify that the provided omitNorms are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameOmitNorms(String fieldName, boolean omitNorms1, boolean omitNorms2) {
+    if (omitNorms1 != omitNorms2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from omitNorms="
+              + omitNorms1
+              + " to inconsistent omitNorms="
+              + omitNorms2);
+    }
+  }
+
+  /**
+   * Verify that the provided points indexing options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySamePointsOptions(

Review comment:
       Addressed in 0fe3493110ac2a5f750ad41f732436daff6c69f5




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org