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/01/07 23:03:58 UTC

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

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


   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
   


----------------------------------------------------------------
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-solr] mayya-sharipova commented on a change in pull request #2186: LUCENE-9334 Consistency of field data structures

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



##########
File path: lucene/test-framework/src/java/org/apache/lucene/index/BaseVectorFormatTestCase.java
##########
@@ -131,7 +115,8 @@ public void testIllegalDimChangeTwoDocs() throws Exception {
       Document doc = new Document();
       doc.add(new VectorField("f", new float[4], VectorValues.SearchStrategy.DOT_PRODUCT_HNSW));
       w.addDocument(doc);
-      if (random().nextBoolean()) {
+      boolean rb = random().nextBoolean();
+      if (rb) {

Review comment:
       Addressed [here](https://github.com/apache/lucene/pull/11/commits/0fe3493110ac2a5f750ad41f732436daff6c69f5#diff-0761a46b20367b4f8c4c85e370dd0156270193e5b897064369cfae68523e81bdL118)




----------------------------------------------------------------
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-solr] jpountz commented on a change in pull request #2186: LUCENE-9334 Consistency of field data structures

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



##########
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(

Review comment:
       the different name suggests that it's checking something different, I think it would be less confusing to use the same method name since this does the same thing?

##########
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);

Review comment:
       this call is a bit weird as the doc values generation is not part of the schema of a field, can you explain?

##########
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) {

Review comment:
       Can you add javadocs or comments about the contract of this function?

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -481,205 +433,102 @@ synchronized int addOrGet(
                 + fieldName
                 + "] as non-soft-deletes already");
       }
-
       return fieldNumber.intValue();
     }
 
-    synchronized void verifyConsistent(Integer number, String name, IndexOptions indexOptions) {
-      if (name.equals(numberToName.get(number)) == false) {
-        throw new IllegalArgumentException(
-            "field number "
-                + number
-                + " is already mapped to field name \""
-                + numberToName.get(number)
-                + "\", not \""
-                + name
-                + "\"");
-      }
-      if (number.equals(nameToNumber.get(name)) == false) {
-        throw new IllegalArgumentException(
-            "field name \""
-                + name
-                + "\" is already mapped to field number \""
-                + nameToNumber.get(name)
-                + "\", not \""
-                + number
-                + "\"");
-      }
-      IndexOptions currentIndexOptions = this.indexOptions.get(name);
-      if (indexOptions != IndexOptions.NONE
-          && currentIndexOptions != null
-          && currentIndexOptions != IndexOptions.NONE
-          && indexOptions != currentIndexOptions) {
-        throw new IllegalArgumentException(
-            "cannot change field \""
-                + name
-                + "\" from index options="
-                + currentIndexOptions
-                + " to inconsistent index options="
-                + indexOptions);
-      }
-    }
-
-    synchronized void verifyConsistent(Integer number, String name, DocValuesType dvType) {
-      if (name.equals(numberToName.get(number)) == false) {
-        throw new IllegalArgumentException(
-            "field number "
-                + number
-                + " is already mapped to field name \""
-                + numberToName.get(number)
-                + "\", not \""
-                + name
-                + "\"");
-      }
-      if (number.equals(nameToNumber.get(name)) == false) {
-        throw new IllegalArgumentException(
-            "field name \""
-                + name
-                + "\" is already mapped to field number \""
-                + nameToNumber.get(name)
-                + "\", not \""
-                + number
-                + "\"");
-      }
-      DocValuesType currentDVType = docValuesType.get(name);
-      if (dvType != DocValuesType.NONE
-          && currentDVType != null
-          && currentDVType != DocValuesType.NONE
-          && dvType != currentDVType) {
-        throw new IllegalArgumentException(
-            "cannot change DocValues type from "
-                + currentDVType
-                + " to "
-                + dvType
-                + " for field \""
-                + name
-                + "\"");
-      }
-    }
-
-    synchronized void verifyConsistentDimensions(
-        Integer number,
-        String name,
-        int dataDimensionCount,
+    private void verifySameSchema(
+        String fieldName,
+        IndexOptions indexOptions,
+        boolean storeTermVector,
+        boolean omitNorms,
+        DocValuesType dvType,
+        int dimensionCount,
         int indexDimensionCount,
-        int dimensionNumBytes) {
-      if (name.equals(numberToName.get(number)) == false) {
-        throw new IllegalArgumentException(
-            "field number "
-                + number
-                + " is already mapped to field name \""
-                + numberToName.get(number)
-                + "\", not \""
-                + name
-                + "\"");
-      }
-      if (number.equals(nameToNumber.get(name)) == false) {
-        throw new IllegalArgumentException(
-            "field name \""
-                + name
-                + "\" is already mapped to field number \""
-                + nameToNumber.get(name)
-                + "\", not \""
-                + number
-                + "\"");
-      }
-      FieldDimensions dim = dimensions.get(name);
-      if (dim != null) {
-        if (dim.dimensionCount != dataDimensionCount) {
-          throw new IllegalArgumentException(
-              "cannot change point dimension count from "
-                  + dim.dimensionCount
-                  + " to "
-                  + dataDimensionCount
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-        if (dim.indexDimensionCount != indexDimensionCount) {
-          throw new IllegalArgumentException(
-              "cannot change point index dimension count from "
-                  + dim.indexDimensionCount
-                  + " to "
-                  + indexDimensionCount
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-        if (dim.dimensionNumBytes != dimensionNumBytes) {
-          throw new IllegalArgumentException(
-              "cannot change point numBytes from "
-                  + dim.dimensionNumBytes
-                  + " to "
-                  + dimensionNumBytes
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-      }
+        int dimensionNumBytes,
+        int vectorDimension,
+        VectorValues.SearchStrategy searchStrategy) {
+
+      IndexOptions currentOpts = this.indexOptions.get(fieldName);
+      verifySameIndexOptions(fieldName, currentOpts, indexOptions);
+      if (currentOpts != IndexOptions.NONE) {
+        boolean curStoreTermVector = this.storeTermVectors.get(fieldName);
+        verifySameStoreTermVectors(fieldName, curStoreTermVector, storeTermVector);
+        boolean curOmitNorms = this.omitNorms.get(fieldName);
+        verifySameOmitNorms(fieldName, curOmitNorms, omitNorms);
+      }
+
+      DocValuesType currentDVType = docValuesType.get(fieldName);
+      verifySameDocValuesType(fieldName, currentDVType, dvType);
+
+      FieldDimensions dims = dimensions.get(fieldName);
+      verifySamePointsOptions(
+          fieldName,
+          dims.dimensionCount,
+          dims.indexDimensionCount,
+          dims.dimensionNumBytes,
+          dimensionCount,
+          indexDimensionCount,
+          dimensionNumBytes);
+
+      FieldVectorProperties props = vectorProps.get(fieldName);
+      verifySameVectorOptions(
+          fieldName, props.numDimensions, props.searchStrategy, vectorDimension, searchStrategy);
     }
 
-    synchronized void verifyConsistentVectorProperties(
-        Integer number,
-        String name,
-        int numDimensions,
-        VectorValues.SearchStrategy searchStrategy) {
-      if (name.equals(numberToName.get(number)) == false) {
-        throw new IllegalArgumentException(
-            "field number "
-                + number
-                + " is already mapped to field name \""
-                + numberToName.get(number)
-                + "\", not \""
-                + name
-                + "\"");
-      }
-      if (number.equals(nameToNumber.get(name)) == false) {
-        throw new IllegalArgumentException(
-            "field name \""
-                + name
-                + "\" is already mapped to field number \""
-                + nameToNumber.get(name)
-                + "\", not \""
-                + number
-                + "\"");
-      }
-      FieldVectorProperties props = vectorProps.get(name);
-      if (props != null) {
-        if (props.numDimensions != numDimensions) {
-          throw new IllegalArgumentException(
-              "cannot change vector dimension from "
-                  + props.numDimensions
-                  + " to "
-                  + numDimensions
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-        if (props.searchStrategy != searchStrategy) {
-          throw new IllegalArgumentException(
-              "cannot change vector search strategy from "
-                  + props.searchStrategy
-                  + " to "
-                  + searchStrategy
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-      }
+    /**
+     * Returns true if the {@code fieldName} exists in the map and is of the same {@code dvType},
+     * and it is docValues only field.
+     */
+    synchronized boolean containsDvOnlyField(String fieldName, DocValuesType dvType) {
+      if (nameToNumber.containsKey(fieldName) == false) return false;
+      if (dvType != docValuesType.get(fieldName)) return false;
+      FieldDimensions fdimensions = dimensions.get(fieldName);
+      if (fdimensions != null && fdimensions.dimensionCount != 0) return false;
+      IndexOptions ioptions = indexOptions.get(fieldName);
+      if (ioptions != null && ioptions != IndexOptions.NONE) return false;
+      FieldVectorProperties fvp = vectorProps.get(fieldName);
+      if (fvp != null && fvp.numDimensions != 0) return false;
+      return true;
     }
 
     /**
-     * Returns true if the {@code fieldName} exists in the map and is of the same {@code dvType}.
+     * Construct a new FieldInfo based on the options in global field numbers This method is not
+     * synchronized as all the options it uses are not modifiable.
+     *
+     * @param fieldName name of the field
+     * @param dvType doc values type
+     * @param newFieldNumber a new field number
+     * @return {@code null} if {@code fieldName} doesn't exist in the map or is not of the same
+     *     {@code dvType} returns a new FieldInfo based based on the options in global field numbers
      */
-    synchronized boolean contains(String fieldName, DocValuesType dvType) {
-      // used by IndexWriter.updateNumericDocValue
-      if (!nameToNumber.containsKey(fieldName)) {
-        return false;
-      } else {
-        // only return true if the field has the same dvType as the requested one
-        return dvType == docValuesType.get(fieldName);
-      }
+    FieldInfo constructFieldInfo(String fieldName, DocValuesType dvType, int newFieldNumber) {
+      Integer fieldNumber;
+      synchronized (this) {
+        fieldNumber = nameToNumber.get(fieldName);
+      }
+      if (fieldNumber == null) return null;
+      DocValuesType dvType0 = docValuesType.get(fieldName);
+      if (dvType != dvType0) return null;
+
+      boolean isSoftDeletesField = fieldName.equals(softDeletesFieldName);
+      FieldDimensions dims = dimensions.get(fieldName);
+      FieldVectorProperties vectors = vectorProps.get(fieldName);

Review comment:
       shouldn't it always be null in practice, as this looks used for doc-value updates? (Or am I missing something?
   same question for dimensions

##########
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(

Review comment:
       Does it need to be public or could we make it pkg-private? (likewise for the similar functions that this class holds)

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -1860,8 +1862,8 @@ public long softUpdateDocument(
    */
   public long updateNumericDocValue(Term term, String field, long value) throws IOException {
     ensureOpen();
-    if (!globalFieldNumberMap.contains(field, DocValuesType.NUMERIC)) {
-      throw new IllegalArgumentException("can only update existing numeric-docvalues fields!");
+    if (globalFieldNumberMap.containsDvOnlyField(field, DocValuesType.NUMERIC) == false) {
+      throw new IllegalArgumentException("can only update existing numeric docvalues only fields!");
     }

Review comment:
       I wonder if we should refactor this method from returning a boolean to throwing the exception so that it could tell what feature exactly the field has that it shouldn't have.

##########
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:
       should it be called just `add` for consistency since other methods are just called `add` or `getOrAdd`?

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexingChain.java
##########
@@ -581,37 +584,59 @@ 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"):

Review comment:
       is this comment no longer applicable?

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexingChain.java
##########
@@ -581,37 +584,59 @@ 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)

Review comment:
       maybe put the comment before the variable declaration again so that it doesn't affect indentation?

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexingChain.java
##########
@@ -79,6 +79,7 @@
 
   // Holds fields seen in each document
   private PerField[] fields = new PerField[1];
+  private PerField[] docFields = new PerField[10];

Review comment:
       should we set a smaller initial size to make sure that we exercise the growing logic in our tests?

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -481,205 +433,102 @@ synchronized int addOrGet(
                 + fieldName
                 + "] as non-soft-deletes already");
       }
-
       return fieldNumber.intValue();
     }
 
-    synchronized void verifyConsistent(Integer number, String name, IndexOptions indexOptions) {
-      if (name.equals(numberToName.get(number)) == false) {
-        throw new IllegalArgumentException(
-            "field number "
-                + number
-                + " is already mapped to field name \""
-                + numberToName.get(number)
-                + "\", not \""
-                + name
-                + "\"");
-      }
-      if (number.equals(nameToNumber.get(name)) == false) {
-        throw new IllegalArgumentException(
-            "field name \""
-                + name
-                + "\" is already mapped to field number \""
-                + nameToNumber.get(name)
-                + "\", not \""
-                + number
-                + "\"");
-      }
-      IndexOptions currentIndexOptions = this.indexOptions.get(name);
-      if (indexOptions != IndexOptions.NONE
-          && currentIndexOptions != null
-          && currentIndexOptions != IndexOptions.NONE
-          && indexOptions != currentIndexOptions) {
-        throw new IllegalArgumentException(
-            "cannot change field \""
-                + name
-                + "\" from index options="
-                + currentIndexOptions
-                + " to inconsistent index options="
-                + indexOptions);
-      }
-    }
-
-    synchronized void verifyConsistent(Integer number, String name, DocValuesType dvType) {
-      if (name.equals(numberToName.get(number)) == false) {
-        throw new IllegalArgumentException(
-            "field number "
-                + number
-                + " is already mapped to field name \""
-                + numberToName.get(number)
-                + "\", not \""
-                + name
-                + "\"");
-      }
-      if (number.equals(nameToNumber.get(name)) == false) {
-        throw new IllegalArgumentException(
-            "field name \""
-                + name
-                + "\" is already mapped to field number \""
-                + nameToNumber.get(name)
-                + "\", not \""
-                + number
-                + "\"");
-      }
-      DocValuesType currentDVType = docValuesType.get(name);
-      if (dvType != DocValuesType.NONE
-          && currentDVType != null
-          && currentDVType != DocValuesType.NONE
-          && dvType != currentDVType) {
-        throw new IllegalArgumentException(
-            "cannot change DocValues type from "
-                + currentDVType
-                + " to "
-                + dvType
-                + " for field \""
-                + name
-                + "\"");
-      }
-    }
-
-    synchronized void verifyConsistentDimensions(
-        Integer number,
-        String name,
-        int dataDimensionCount,
+    private void verifySameSchema(
+        String fieldName,
+        IndexOptions indexOptions,
+        boolean storeTermVector,
+        boolean omitNorms,
+        DocValuesType dvType,
+        int dimensionCount,
         int indexDimensionCount,
-        int dimensionNumBytes) {
-      if (name.equals(numberToName.get(number)) == false) {
-        throw new IllegalArgumentException(
-            "field number "
-                + number
-                + " is already mapped to field name \""
-                + numberToName.get(number)
-                + "\", not \""
-                + name
-                + "\"");
-      }
-      if (number.equals(nameToNumber.get(name)) == false) {
-        throw new IllegalArgumentException(
-            "field name \""
-                + name
-                + "\" is already mapped to field number \""
-                + nameToNumber.get(name)
-                + "\", not \""
-                + number
-                + "\"");
-      }
-      FieldDimensions dim = dimensions.get(name);
-      if (dim != null) {
-        if (dim.dimensionCount != dataDimensionCount) {
-          throw new IllegalArgumentException(
-              "cannot change point dimension count from "
-                  + dim.dimensionCount
-                  + " to "
-                  + dataDimensionCount
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-        if (dim.indexDimensionCount != indexDimensionCount) {
-          throw new IllegalArgumentException(
-              "cannot change point index dimension count from "
-                  + dim.indexDimensionCount
-                  + " to "
-                  + indexDimensionCount
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-        if (dim.dimensionNumBytes != dimensionNumBytes) {
-          throw new IllegalArgumentException(
-              "cannot change point numBytes from "
-                  + dim.dimensionNumBytes
-                  + " to "
-                  + dimensionNumBytes
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-      }
+        int dimensionNumBytes,
+        int vectorDimension,
+        VectorValues.SearchStrategy searchStrategy) {
+
+      IndexOptions currentOpts = this.indexOptions.get(fieldName);
+      verifySameIndexOptions(fieldName, currentOpts, indexOptions);
+      if (currentOpts != IndexOptions.NONE) {
+        boolean curStoreTermVector = this.storeTermVectors.get(fieldName);
+        verifySameStoreTermVectors(fieldName, curStoreTermVector, storeTermVector);
+        boolean curOmitNorms = this.omitNorms.get(fieldName);
+        verifySameOmitNorms(fieldName, curOmitNorms, omitNorms);
+      }
+
+      DocValuesType currentDVType = docValuesType.get(fieldName);
+      verifySameDocValuesType(fieldName, currentDVType, dvType);
+
+      FieldDimensions dims = dimensions.get(fieldName);
+      verifySamePointsOptions(
+          fieldName,
+          dims.dimensionCount,
+          dims.indexDimensionCount,
+          dims.dimensionNumBytes,
+          dimensionCount,
+          indexDimensionCount,
+          dimensionNumBytes);
+
+      FieldVectorProperties props = vectorProps.get(fieldName);
+      verifySameVectorOptions(
+          fieldName, props.numDimensions, props.searchStrategy, vectorDimension, searchStrategy);
     }
 
-    synchronized void verifyConsistentVectorProperties(
-        Integer number,
-        String name,
-        int numDimensions,
-        VectorValues.SearchStrategy searchStrategy) {
-      if (name.equals(numberToName.get(number)) == false) {
-        throw new IllegalArgumentException(
-            "field number "
-                + number
-                + " is already mapped to field name \""
-                + numberToName.get(number)
-                + "\", not \""
-                + name
-                + "\"");
-      }
-      if (number.equals(nameToNumber.get(name)) == false) {
-        throw new IllegalArgumentException(
-            "field name \""
-                + name
-                + "\" is already mapped to field number \""
-                + nameToNumber.get(name)
-                + "\", not \""
-                + number
-                + "\"");
-      }
-      FieldVectorProperties props = vectorProps.get(name);
-      if (props != null) {
-        if (props.numDimensions != numDimensions) {
-          throw new IllegalArgumentException(
-              "cannot change vector dimension from "
-                  + props.numDimensions
-                  + " to "
-                  + numDimensions
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-        if (props.searchStrategy != searchStrategy) {
-          throw new IllegalArgumentException(
-              "cannot change vector search strategy from "
-                  + props.searchStrategy
-                  + " to "
-                  + searchStrategy
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-      }
+    /**
+     * Returns true if the {@code fieldName} exists in the map and is of the same {@code dvType},
+     * and it is docValues only field.
+     */
+    synchronized boolean containsDvOnlyField(String fieldName, DocValuesType dvType) {
+      if (nameToNumber.containsKey(fieldName) == false) return false;
+      if (dvType != docValuesType.get(fieldName)) return false;
+      FieldDimensions fdimensions = dimensions.get(fieldName);
+      if (fdimensions != null && fdimensions.dimensionCount != 0) return false;
+      IndexOptions ioptions = indexOptions.get(fieldName);
+      if (ioptions != null && ioptions != IndexOptions.NONE) return false;
+      FieldVectorProperties fvp = vectorProps.get(fieldName);
+      if (fvp != null && fvp.numDimensions != 0) return false;
+      return true;
     }
 
     /**
-     * Returns true if the {@code fieldName} exists in the map and is of the same {@code dvType}.
+     * Construct a new FieldInfo based on the options in global field numbers This method is not
+     * synchronized as all the options it uses are not modifiable.

Review comment:
       is it correct? it looks to me like `docValuesType` could be modified by another thread concurrently to the current thread reading from it through this method?

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -455,8 +373,42 @@ synchronized int addOrGet(
           fieldNumber = lowestUnassignedFieldNumber;
         }
         assert fieldNumber >= 0;
+        FieldInfo.checkOptionsCorrectness(
+            fieldName,
+            false,

Review comment:
       why are we not passing `storeTermVector` here?

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -481,205 +433,102 @@ synchronized int addOrGet(
                 + fieldName
                 + "] as non-soft-deletes already");
       }
-
       return fieldNumber.intValue();
     }
 
-    synchronized void verifyConsistent(Integer number, String name, IndexOptions indexOptions) {
-      if (name.equals(numberToName.get(number)) == false) {
-        throw new IllegalArgumentException(
-            "field number "
-                + number
-                + " is already mapped to field name \""
-                + numberToName.get(number)
-                + "\", not \""
-                + name
-                + "\"");
-      }
-      if (number.equals(nameToNumber.get(name)) == false) {
-        throw new IllegalArgumentException(
-            "field name \""
-                + name
-                + "\" is already mapped to field number \""
-                + nameToNumber.get(name)
-                + "\", not \""
-                + number
-                + "\"");
-      }
-      IndexOptions currentIndexOptions = this.indexOptions.get(name);
-      if (indexOptions != IndexOptions.NONE
-          && currentIndexOptions != null
-          && currentIndexOptions != IndexOptions.NONE
-          && indexOptions != currentIndexOptions) {
-        throw new IllegalArgumentException(
-            "cannot change field \""
-                + name
-                + "\" from index options="
-                + currentIndexOptions
-                + " to inconsistent index options="
-                + indexOptions);
-      }
-    }
-
-    synchronized void verifyConsistent(Integer number, String name, DocValuesType dvType) {
-      if (name.equals(numberToName.get(number)) == false) {
-        throw new IllegalArgumentException(
-            "field number "
-                + number
-                + " is already mapped to field name \""
-                + numberToName.get(number)
-                + "\", not \""
-                + name
-                + "\"");
-      }
-      if (number.equals(nameToNumber.get(name)) == false) {
-        throw new IllegalArgumentException(
-            "field name \""
-                + name
-                + "\" is already mapped to field number \""
-                + nameToNumber.get(name)
-                + "\", not \""
-                + number
-                + "\"");
-      }
-      DocValuesType currentDVType = docValuesType.get(name);
-      if (dvType != DocValuesType.NONE
-          && currentDVType != null
-          && currentDVType != DocValuesType.NONE
-          && dvType != currentDVType) {
-        throw new IllegalArgumentException(
-            "cannot change DocValues type from "
-                + currentDVType
-                + " to "
-                + dvType
-                + " for field \""
-                + name
-                + "\"");
-      }
-    }
-
-    synchronized void verifyConsistentDimensions(
-        Integer number,
-        String name,
-        int dataDimensionCount,
+    private void verifySameSchema(
+        String fieldName,
+        IndexOptions indexOptions,
+        boolean storeTermVector,
+        boolean omitNorms,
+        DocValuesType dvType,
+        int dimensionCount,
         int indexDimensionCount,
-        int dimensionNumBytes) {
-      if (name.equals(numberToName.get(number)) == false) {
-        throw new IllegalArgumentException(
-            "field number "
-                + number
-                + " is already mapped to field name \""
-                + numberToName.get(number)
-                + "\", not \""
-                + name
-                + "\"");
-      }
-      if (number.equals(nameToNumber.get(name)) == false) {
-        throw new IllegalArgumentException(
-            "field name \""
-                + name
-                + "\" is already mapped to field number \""
-                + nameToNumber.get(name)
-                + "\", not \""
-                + number
-                + "\"");
-      }
-      FieldDimensions dim = dimensions.get(name);
-      if (dim != null) {
-        if (dim.dimensionCount != dataDimensionCount) {
-          throw new IllegalArgumentException(
-              "cannot change point dimension count from "
-                  + dim.dimensionCount
-                  + " to "
-                  + dataDimensionCount
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-        if (dim.indexDimensionCount != indexDimensionCount) {
-          throw new IllegalArgumentException(
-              "cannot change point index dimension count from "
-                  + dim.indexDimensionCount
-                  + " to "
-                  + indexDimensionCount
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-        if (dim.dimensionNumBytes != dimensionNumBytes) {
-          throw new IllegalArgumentException(
-              "cannot change point numBytes from "
-                  + dim.dimensionNumBytes
-                  + " to "
-                  + dimensionNumBytes
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-      }
+        int dimensionNumBytes,
+        int vectorDimension,
+        VectorValues.SearchStrategy searchStrategy) {
+
+      IndexOptions currentOpts = this.indexOptions.get(fieldName);
+      verifySameIndexOptions(fieldName, currentOpts, indexOptions);
+      if (currentOpts != IndexOptions.NONE) {
+        boolean curStoreTermVector = this.storeTermVectors.get(fieldName);
+        verifySameStoreTermVectors(fieldName, curStoreTermVector, storeTermVector);
+        boolean curOmitNorms = this.omitNorms.get(fieldName);
+        verifySameOmitNorms(fieldName, curOmitNorms, omitNorms);
+      }
+
+      DocValuesType currentDVType = docValuesType.get(fieldName);
+      verifySameDocValuesType(fieldName, currentDVType, dvType);
+
+      FieldDimensions dims = dimensions.get(fieldName);
+      verifySamePointsOptions(
+          fieldName,
+          dims.dimensionCount,
+          dims.indexDimensionCount,
+          dims.dimensionNumBytes,
+          dimensionCount,
+          indexDimensionCount,
+          dimensionNumBytes);
+
+      FieldVectorProperties props = vectorProps.get(fieldName);
+      verifySameVectorOptions(
+          fieldName, props.numDimensions, props.searchStrategy, vectorDimension, searchStrategy);
     }
 
-    synchronized void verifyConsistentVectorProperties(
-        Integer number,
-        String name,
-        int numDimensions,
-        VectorValues.SearchStrategy searchStrategy) {
-      if (name.equals(numberToName.get(number)) == false) {
-        throw new IllegalArgumentException(
-            "field number "
-                + number
-                + " is already mapped to field name \""
-                + numberToName.get(number)
-                + "\", not \""
-                + name
-                + "\"");
-      }
-      if (number.equals(nameToNumber.get(name)) == false) {
-        throw new IllegalArgumentException(
-            "field name \""
-                + name
-                + "\" is already mapped to field number \""
-                + nameToNumber.get(name)
-                + "\", not \""
-                + number
-                + "\"");
-      }
-      FieldVectorProperties props = vectorProps.get(name);
-      if (props != null) {
-        if (props.numDimensions != numDimensions) {
-          throw new IllegalArgumentException(
-              "cannot change vector dimension from "
-                  + props.numDimensions
-                  + " to "
-                  + numDimensions
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-        if (props.searchStrategy != searchStrategy) {
-          throw new IllegalArgumentException(
-              "cannot change vector search strategy from "
-                  + props.searchStrategy
-                  + " to "
-                  + searchStrategy
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-      }
+    /**
+     * Returns true if the {@code fieldName} exists in the map and is of the same {@code dvType},
+     * and it is docValues only field.

Review comment:
       I think we could use more docs here, e.g. explaining that this is called from IndexWriter to know whether a field may be updated through doc values?

##########
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:
       Can you add javadocs since it's public?

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexingChain.java
##########
@@ -399,7 +401,8 @@ private void writeDocValues(SegmentWriteState state, Sorter.DocMap sortMap) thro
             }
             perField.docValuesWriter.flush(state, sortMap, dvConsumer);
             perField.docValuesWriter = null;
-          } else if (perField.fieldInfo.getDocValuesType() != DocValuesType.NONE) {
+          } else if (perField.fieldInfo != null

Review comment:
       shouldn't fieldInfo always be non null when reaching here?

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexingChain.java
##########
@@ -349,7 +350,8 @@ private void writePoints(SegmentWriteState state, Sorter.DocMap sortMap) throws
 
             perField.pointValuesWriter.flush(state, sortMap, pointsWriter);
             perField.pointValuesWriter = null;
-          } else if (perField.fieldInfo.getPointDimensionCount() != 0) {
+          } else if (perField.fieldInfo != null

Review comment:
       shouldn't fieldInfo always be non null when reaching here?




----------------------------------------------------------------
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-solr] jpountz commented on pull request #2186: LUCENE-9334 Consistency of field data structures

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


   I need to spend more time understanding how everything ties together but the logic you described makes sense to me.


----------------------------------------------------------------
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-solr] mayya-sharipova commented on a change in pull request #2186: LUCENE-9334 Consistency of field data structures

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



##########
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:
       Addressed [here](https://github.com/apache/lucene/pull/11/commits/0fe3493110ac2a5f750ad41f732436daff6c69f5#diff-b81932cf39b70347481b9659dd55b9b373427c2992cdacdfbc10ff8f4ea9eec0R745)




----------------------------------------------------------------
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-solr] mayya-sharipova commented on pull request #2186: LUCENE-9334 Consistency of field data structures

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


   > Could we make this work across segments?
   
   @jpountz  Thanks for taking a look. I just wanted to confirm that we are ok with this change, which means for example, that we can't enable  doc_values on a field where doc values were previously disabled. 


----------------------------------------------------------------
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-solr] mayya-sharipova commented on a change in pull request #2186: LUCENE-9334 Consistency of field data structures

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



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexingChain.java
##########
@@ -581,37 +584,59 @@ 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"):

Review comment:
       I am not sure, as I did not study deeply of termHash. But just in case, I've added this comment back [here](https://github.com/apache/lucene/pull/11/commits/0fe3493110ac2a5f750ad41f732436daff6c69f5#diff-1e11574161aacb332207eea2eb36edbf86bf3518416e4c089561f3c0fbe43e03R593)




----------------------------------------------------------------
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-solr] mayya-sharipova commented on pull request #2186: LUCENE-9334 Consistency of field data structures

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


   @jpountz Thanks a lot for the second round of review. I've addressed these comments in the [new repo](https://github.com/apache/lucene/pull/11/commits/0fe3493110ac2a5f750ad41f732436daff6c69f5).
   
   I've resolved all comments that I fully addressed, and left comments on some that need more discussion. 


----------------------------------------------------------------
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-solr] mayya-sharipova commented on a change in pull request #2186: LUCENE-9334 Consistency of field data structures

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



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -481,205 +433,102 @@ synchronized int addOrGet(
                 + fieldName
                 + "] as non-soft-deletes already");
       }
-
       return fieldNumber.intValue();
     }
 
-    synchronized void verifyConsistent(Integer number, String name, IndexOptions indexOptions) {
-      if (name.equals(numberToName.get(number)) == false) {
-        throw new IllegalArgumentException(
-            "field number "
-                + number
-                + " is already mapped to field name \""
-                + numberToName.get(number)
-                + "\", not \""
-                + name
-                + "\"");
-      }
-      if (number.equals(nameToNumber.get(name)) == false) {
-        throw new IllegalArgumentException(
-            "field name \""
-                + name
-                + "\" is already mapped to field number \""
-                + nameToNumber.get(name)
-                + "\", not \""
-                + number
-                + "\"");
-      }
-      IndexOptions currentIndexOptions = this.indexOptions.get(name);
-      if (indexOptions != IndexOptions.NONE
-          && currentIndexOptions != null
-          && currentIndexOptions != IndexOptions.NONE
-          && indexOptions != currentIndexOptions) {
-        throw new IllegalArgumentException(
-            "cannot change field \""
-                + name
-                + "\" from index options="
-                + currentIndexOptions
-                + " to inconsistent index options="
-                + indexOptions);
-      }
-    }
-
-    synchronized void verifyConsistent(Integer number, String name, DocValuesType dvType) {
-      if (name.equals(numberToName.get(number)) == false) {
-        throw new IllegalArgumentException(
-            "field number "
-                + number
-                + " is already mapped to field name \""
-                + numberToName.get(number)
-                + "\", not \""
-                + name
-                + "\"");
-      }
-      if (number.equals(nameToNumber.get(name)) == false) {
-        throw new IllegalArgumentException(
-            "field name \""
-                + name
-                + "\" is already mapped to field number \""
-                + nameToNumber.get(name)
-                + "\", not \""
-                + number
-                + "\"");
-      }
-      DocValuesType currentDVType = docValuesType.get(name);
-      if (dvType != DocValuesType.NONE
-          && currentDVType != null
-          && currentDVType != DocValuesType.NONE
-          && dvType != currentDVType) {
-        throw new IllegalArgumentException(
-            "cannot change DocValues type from "
-                + currentDVType
-                + " to "
-                + dvType
-                + " for field \""
-                + name
-                + "\"");
-      }
-    }
-
-    synchronized void verifyConsistentDimensions(
-        Integer number,
-        String name,
-        int dataDimensionCount,
+    private void verifySameSchema(
+        String fieldName,
+        IndexOptions indexOptions,
+        boolean storeTermVector,
+        boolean omitNorms,
+        DocValuesType dvType,
+        int dimensionCount,
         int indexDimensionCount,
-        int dimensionNumBytes) {
-      if (name.equals(numberToName.get(number)) == false) {
-        throw new IllegalArgumentException(
-            "field number "
-                + number
-                + " is already mapped to field name \""
-                + numberToName.get(number)
-                + "\", not \""
-                + name
-                + "\"");
-      }
-      if (number.equals(nameToNumber.get(name)) == false) {
-        throw new IllegalArgumentException(
-            "field name \""
-                + name
-                + "\" is already mapped to field number \""
-                + nameToNumber.get(name)
-                + "\", not \""
-                + number
-                + "\"");
-      }
-      FieldDimensions dim = dimensions.get(name);
-      if (dim != null) {
-        if (dim.dimensionCount != dataDimensionCount) {
-          throw new IllegalArgumentException(
-              "cannot change point dimension count from "
-                  + dim.dimensionCount
-                  + " to "
-                  + dataDimensionCount
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-        if (dim.indexDimensionCount != indexDimensionCount) {
-          throw new IllegalArgumentException(
-              "cannot change point index dimension count from "
-                  + dim.indexDimensionCount
-                  + " to "
-                  + indexDimensionCount
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-        if (dim.dimensionNumBytes != dimensionNumBytes) {
-          throw new IllegalArgumentException(
-              "cannot change point numBytes from "
-                  + dim.dimensionNumBytes
-                  + " to "
-                  + dimensionNumBytes
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-      }
+        int dimensionNumBytes,
+        int vectorDimension,
+        VectorValues.SearchStrategy searchStrategy) {
+
+      IndexOptions currentOpts = this.indexOptions.get(fieldName);
+      verifySameIndexOptions(fieldName, currentOpts, indexOptions);
+      if (currentOpts != IndexOptions.NONE) {
+        boolean curStoreTermVector = this.storeTermVectors.get(fieldName);
+        verifySameStoreTermVectors(fieldName, curStoreTermVector, storeTermVector);
+        boolean curOmitNorms = this.omitNorms.get(fieldName);
+        verifySameOmitNorms(fieldName, curOmitNorms, omitNorms);
+      }
+
+      DocValuesType currentDVType = docValuesType.get(fieldName);
+      verifySameDocValuesType(fieldName, currentDVType, dvType);
+
+      FieldDimensions dims = dimensions.get(fieldName);
+      verifySamePointsOptions(
+          fieldName,
+          dims.dimensionCount,
+          dims.indexDimensionCount,
+          dims.dimensionNumBytes,
+          dimensionCount,
+          indexDimensionCount,
+          dimensionNumBytes);
+
+      FieldVectorProperties props = vectorProps.get(fieldName);
+      verifySameVectorOptions(
+          fieldName, props.numDimensions, props.searchStrategy, vectorDimension, searchStrategy);
     }
 
-    synchronized void verifyConsistentVectorProperties(
-        Integer number,
-        String name,
-        int numDimensions,
-        VectorValues.SearchStrategy searchStrategy) {
-      if (name.equals(numberToName.get(number)) == false) {
-        throw new IllegalArgumentException(
-            "field number "
-                + number
-                + " is already mapped to field name \""
-                + numberToName.get(number)
-                + "\", not \""
-                + name
-                + "\"");
-      }
-      if (number.equals(nameToNumber.get(name)) == false) {
-        throw new IllegalArgumentException(
-            "field name \""
-                + name
-                + "\" is already mapped to field number \""
-                + nameToNumber.get(name)
-                + "\", not \""
-                + number
-                + "\"");
-      }
-      FieldVectorProperties props = vectorProps.get(name);
-      if (props != null) {
-        if (props.numDimensions != numDimensions) {
-          throw new IllegalArgumentException(
-              "cannot change vector dimension from "
-                  + props.numDimensions
-                  + " to "
-                  + numDimensions
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-        if (props.searchStrategy != searchStrategy) {
-          throw new IllegalArgumentException(
-              "cannot change vector search strategy from "
-                  + props.searchStrategy
-                  + " to "
-                  + searchStrategy
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-      }
+    /**
+     * Returns true if the {@code fieldName} exists in the map and is of the same {@code dvType},
+     * and it is docValues only field.

Review comment:
       Addressed [here](https://github.com/apache/lucene/pull/11/commits/0fe3493110ac2a5f750ad41f732436daff6c69f5#diff-b81932cf39b70347481b9659dd55b9b373427c2992cdacdfbc10ff8f4ea9eec0L479)




----------------------------------------------------------------
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-solr] jpountz commented on pull request #2186: LUCENE-9334 Consistency of field data structures

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


   > 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.
   
   Could we make this work across segments?


----------------------------------------------------------------
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-solr] mayya-sharipova commented on a change in pull request #2186: LUCENE-9334 Consistency of field data structures

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



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexingChain.java
##########
@@ -581,37 +584,59 @@ 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)

Review comment:
       Addressed [here](https://github.com/apache/lucene/pull/11/commits/0fe3493110ac2a5f750ad41f732436daff6c69f5#diff-1e11574161aacb332207eea2eb36edbf86bf3518416e4c089561f3c0fbe43e03L587)




----------------------------------------------------------------
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-solr] mayya-sharipova edited a comment on pull request #2186: LUCENE-9334 Consistency of field data structures

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


   @jpountz  Adrien, I also wanted to ask your opinion about the behaviour of `writer.updateDocValues(Term term, fieldX)`, as I am not sure what to do in this case:
   1.  What should happen if `fieldX` doesn't exist in the segment (but exists globally)? 
   2. What should happen if `fieldX` doesn't have docValues enabled?
   
   Before in both of these cases `updateDocValues` worked well creating a new `FieldInfo` in the current segment if it doesn't exist or enabling doc values.
   
   But with current PR as the field schema is fixed, we can:
   1. Thrown an  error? Or allow only in a case where the global `fieldX`  is only a docValues field?
   2. Throw an error, as field schema is fixed now and  docValues can't be enabled.
   
   WDYT?


----------------------------------------------------------------
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-solr] mayya-sharipova commented on a change in pull request #2186: LUCENE-9334 Consistency of field data structures

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



##########
File path: lucene/test-framework/src/java/org/apache/lucene/index/BasePointsFormatTestCase.java
##########
@@ -1174,8 +1174,11 @@ public void testMixedSchema() throws Exception {
 
     Document doc = new Document();
     doc.add(new IntPoint("id", 0));
-    w.addDocument(doc);
-    // now we write another segment where the id field does have points:
+    IllegalArgumentException ex =
+        expectThrows(IllegalArgumentException.class, () -> w.addDocument(doc));
+    assertEquals(
+        "cannot change field \"id\" from index options=DOCS to inconsistent index options=NONE",
+        ex.getMessage());

Review comment:
       The test refactored [here](https://github.com/apache/lucene/pull/11/commits/0fe3493110ac2a5f750ad41f732436daff6c69f5#diff-974afcef788664579ec220b758dd40bf67d8e72f3fb214e52016b8465fb54289L1161)




----------------------------------------------------------------
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-solr] mayya-sharipova commented on a change in pull request #2186: LUCENE-9334 Consistency of field data structures

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



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexingChain.java
##########
@@ -79,6 +79,7 @@
 
   // Holds fields seen in each document
   private PerField[] fields = new PerField[1];
+  private PerField[] docFields = new PerField[10];

Review comment:
       @jpountz  I wonder what should be a smaller initial size for the number of fields in a doc?
   
   I changed the size to 2 [here](https://github.com/apache/lucene/pull/11/commits/0fe3493110ac2a5f750ad41f732436daff6c69f5#diff-1e11574161aacb332207eea2eb36edbf86bf3518416e4c089561f3c0fbe43e03L82)




----------------------------------------------------------------
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-solr] mayya-sharipova commented on a change in pull request #2186: LUCENE-9334 Consistency of field data structures

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



##########
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);

Review comment:
       Addressed [here](https://github.com/apache/lucene/pull/11/commits/0fe3493110ac2a5f750ad41f732436daff6c69f5#diff-b9ceb6f65b3597b6c2406c2d41cc7db39e04e21bb77a13fcc92eb0429c444d46L252)




----------------------------------------------------------------
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-solr] mayya-sharipova commented on a change in pull request #2186: LUCENE-9334 Consistency of field data structures

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



##########
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(

Review comment:
       Addressed [here](https://github.com/apache/lucene/pull/11/commits/0fe3493110ac2a5f750ad41f732436daff6c69f5#diff-b9ceb6f65b3597b6c2406c2d41cc7db39e04e21bb77a13fcc92eb0429c444d46L274)




----------------------------------------------------------------
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-solr] mayya-sharipova commented on a change in pull request #2186: LUCENE-9334 Consistency of field data structures

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



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexingChain.java
##########
@@ -399,7 +401,8 @@ private void writeDocValues(SegmentWriteState state, Sorter.DocMap sortMap) thro
             }
             perField.docValuesWriter.flush(state, sortMap, dvConsumer);
             perField.docValuesWriter = null;
-          } else if (perField.fieldInfo.getDocValuesType() != DocValuesType.NONE) {
+          } else if (perField.fieldInfo != null

Review comment:
       Not, always.
   We first create PerField: `PerField pf = getOrAddPerField(field.name(), fieldType);` and then initialize its FieldInfo `initializeFieldInfo(pf)`. Initialization of FieldInfo may fail, if there is already a global field with the same name but a different schema. So we may end up with `PerField` with a null `fieldInfo`.




----------------------------------------------------------------
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-solr] mayya-sharipova commented on a change in pull request #2186: LUCENE-9334 Consistency of field data structures

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



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -130,127 +167,255 @@ 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,
+  void verifySameSchema(
+      IndexOptions indexOptions,
       boolean omitNorms,
       boolean storePayloads,
-      IndexOptions indexOptions,
-      Map<String, String> attributes,
+      boolean storeTermVector,
+      DocValuesType docValuesType,
+      long dvGen,
       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);
-      }
+      int dimensionNumBytes,
+      int vectorDimension,
+      VectorValues.SearchStrategy searchStrategy) {
+    verifySameIndexOptions(this.name, this.indexOptions, indexOptions);
+    if (this.indexOptions != IndexOptions.NONE) {
+      verifySamePostingsOptions(
+          this.name,
+          this.storeTermVector,
+          this.omitNorms,
+          this.storePayloads,
+          storeTermVector,
+          omitNorms,
+          storePayloads);
+    }
+    verifySameDocValuesType(this.name, this.docValuesType, docValuesType);
+    verifySameDVGen(this.name, this.dvGen, dvGen);
+    verifySamePointsOptions(
+        this.name,
+        this.pointDimensionCount,
+        this.pointIndexDimensionCount,
+        this.pointNumBytes,
+        dimensionCount,
+        indexDimensionCount,
+        dimensionNumBytes);
+    verifySameVectorOptions(
+        this.name,
+        this.vectorDimension,
+        this.vectorSearchStrategy,
+        vectorDimension,
+        searchStrategy);
+  }
+
+  /**
+   * Very that the provided index options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameIndexOptions(String fieldName, IndexOptions io1, IndexOptions io2) {
+    if (io1 != io2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from index options="
+              + io1
+              + " to inconsistent index options="
+              + io2);
     }
+  }
 
-    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)) {
+  /**
+   * Very that the provided docValues type are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameDocValuesType(
+      String fieldName, DocValuesType dv1, DocValuesType dv2) {
+    if (dv1 != dv2) {
       throw new IllegalArgumentException(
           "cannot change field \""
-              + name
+              + fieldName
+              + "\" from doc values type="
+              + dv1
+              + " to inconsistent doc values type="
+              + dv2);
+    }
+  }
+
+  /**
+   * Very that the provided doc values generations are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameDVGen(String fieldName, long dvg1, long dvg2) {
+    if (dvg1 != dvg2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from doc values generation="
+              + dvg1
+              + " to inconsistent doc values generation="
+              + dvg2);
+    }
+  }
+
+  /**
+   * Very that the provided posting options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySamePostingsOptions(
+      String fieldName,
+      boolean stv1,
+      boolean on1,
+      boolean sp1,
+      boolean stv2,
+      boolean on2,
+      boolean sp2) {
+    if (stv1 != stv2 || on1 != on2 || sp1 != sp2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from storeTermVector="
+              + stv1
+              + ", omitNorms="
+              + on1
+              + ", storePayloads="
+              + sp1
+              + " to inconsistent storeTermVector="
+              + stv2
+              + ", omitNorms="
+              + on2
+              + ", storePayloads="
+              + sp2);
+    }
+  }
+
+  /**
+   * Very that the provided points indexing options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySamePointsOptions(
+      String fieldName, int dc1, int idc1, int nb1, int dc2, int idc2, int nb2) {
+    if (dc1 != dc2 || idc1 != idc2 || nb1 != nb2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
               + "\" from points dimensionCount="
-              + this.pointDimensionCount
+              + dc1

Review comment:
       Addressed in 6e7540ebd0ef79536cffabcf0ddc7a592b792252




----------------------------------------------------------------
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-solr] mayya-sharipova commented on a change in pull request #2186: LUCENE-9334 Consistency of field data structures

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



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -481,205 +433,102 @@ synchronized int addOrGet(
                 + fieldName
                 + "] as non-soft-deletes already");
       }
-
       return fieldNumber.intValue();
     }
 
-    synchronized void verifyConsistent(Integer number, String name, IndexOptions indexOptions) {
-      if (name.equals(numberToName.get(number)) == false) {
-        throw new IllegalArgumentException(
-            "field number "
-                + number
-                + " is already mapped to field name \""
-                + numberToName.get(number)
-                + "\", not \""
-                + name
-                + "\"");
-      }
-      if (number.equals(nameToNumber.get(name)) == false) {
-        throw new IllegalArgumentException(
-            "field name \""
-                + name
-                + "\" is already mapped to field number \""
-                + nameToNumber.get(name)
-                + "\", not \""
-                + number
-                + "\"");
-      }
-      IndexOptions currentIndexOptions = this.indexOptions.get(name);
-      if (indexOptions != IndexOptions.NONE
-          && currentIndexOptions != null
-          && currentIndexOptions != IndexOptions.NONE
-          && indexOptions != currentIndexOptions) {
-        throw new IllegalArgumentException(
-            "cannot change field \""
-                + name
-                + "\" from index options="
-                + currentIndexOptions
-                + " to inconsistent index options="
-                + indexOptions);
-      }
-    }
-
-    synchronized void verifyConsistent(Integer number, String name, DocValuesType dvType) {
-      if (name.equals(numberToName.get(number)) == false) {
-        throw new IllegalArgumentException(
-            "field number "
-                + number
-                + " is already mapped to field name \""
-                + numberToName.get(number)
-                + "\", not \""
-                + name
-                + "\"");
-      }
-      if (number.equals(nameToNumber.get(name)) == false) {
-        throw new IllegalArgumentException(
-            "field name \""
-                + name
-                + "\" is already mapped to field number \""
-                + nameToNumber.get(name)
-                + "\", not \""
-                + number
-                + "\"");
-      }
-      DocValuesType currentDVType = docValuesType.get(name);
-      if (dvType != DocValuesType.NONE
-          && currentDVType != null
-          && currentDVType != DocValuesType.NONE
-          && dvType != currentDVType) {
-        throw new IllegalArgumentException(
-            "cannot change DocValues type from "
-                + currentDVType
-                + " to "
-                + dvType
-                + " for field \""
-                + name
-                + "\"");
-      }
-    }
-
-    synchronized void verifyConsistentDimensions(
-        Integer number,
-        String name,
-        int dataDimensionCount,
+    private void verifySameSchema(
+        String fieldName,
+        IndexOptions indexOptions,
+        boolean storeTermVector,
+        boolean omitNorms,
+        DocValuesType dvType,
+        int dimensionCount,
         int indexDimensionCount,
-        int dimensionNumBytes) {
-      if (name.equals(numberToName.get(number)) == false) {
-        throw new IllegalArgumentException(
-            "field number "
-                + number
-                + " is already mapped to field name \""
-                + numberToName.get(number)
-                + "\", not \""
-                + name
-                + "\"");
-      }
-      if (number.equals(nameToNumber.get(name)) == false) {
-        throw new IllegalArgumentException(
-            "field name \""
-                + name
-                + "\" is already mapped to field number \""
-                + nameToNumber.get(name)
-                + "\", not \""
-                + number
-                + "\"");
-      }
-      FieldDimensions dim = dimensions.get(name);
-      if (dim != null) {
-        if (dim.dimensionCount != dataDimensionCount) {
-          throw new IllegalArgumentException(
-              "cannot change point dimension count from "
-                  + dim.dimensionCount
-                  + " to "
-                  + dataDimensionCount
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-        if (dim.indexDimensionCount != indexDimensionCount) {
-          throw new IllegalArgumentException(
-              "cannot change point index dimension count from "
-                  + dim.indexDimensionCount
-                  + " to "
-                  + indexDimensionCount
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-        if (dim.dimensionNumBytes != dimensionNumBytes) {
-          throw new IllegalArgumentException(
-              "cannot change point numBytes from "
-                  + dim.dimensionNumBytes
-                  + " to "
-                  + dimensionNumBytes
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-      }
+        int dimensionNumBytes,
+        int vectorDimension,
+        VectorValues.SearchStrategy searchStrategy) {
+
+      IndexOptions currentOpts = this.indexOptions.get(fieldName);
+      verifySameIndexOptions(fieldName, currentOpts, indexOptions);
+      if (currentOpts != IndexOptions.NONE) {
+        boolean curStoreTermVector = this.storeTermVectors.get(fieldName);
+        verifySameStoreTermVectors(fieldName, curStoreTermVector, storeTermVector);
+        boolean curOmitNorms = this.omitNorms.get(fieldName);
+        verifySameOmitNorms(fieldName, curOmitNorms, omitNorms);
+      }
+
+      DocValuesType currentDVType = docValuesType.get(fieldName);
+      verifySameDocValuesType(fieldName, currentDVType, dvType);
+
+      FieldDimensions dims = dimensions.get(fieldName);
+      verifySamePointsOptions(
+          fieldName,
+          dims.dimensionCount,
+          dims.indexDimensionCount,
+          dims.dimensionNumBytes,
+          dimensionCount,
+          indexDimensionCount,
+          dimensionNumBytes);
+
+      FieldVectorProperties props = vectorProps.get(fieldName);
+      verifySameVectorOptions(
+          fieldName, props.numDimensions, props.searchStrategy, vectorDimension, searchStrategy);
     }
 
-    synchronized void verifyConsistentVectorProperties(
-        Integer number,
-        String name,
-        int numDimensions,
-        VectorValues.SearchStrategy searchStrategy) {
-      if (name.equals(numberToName.get(number)) == false) {
-        throw new IllegalArgumentException(
-            "field number "
-                + number
-                + " is already mapped to field name \""
-                + numberToName.get(number)
-                + "\", not \""
-                + name
-                + "\"");
-      }
-      if (number.equals(nameToNumber.get(name)) == false) {
-        throw new IllegalArgumentException(
-            "field name \""
-                + name
-                + "\" is already mapped to field number \""
-                + nameToNumber.get(name)
-                + "\", not \""
-                + number
-                + "\"");
-      }
-      FieldVectorProperties props = vectorProps.get(name);
-      if (props != null) {
-        if (props.numDimensions != numDimensions) {
-          throw new IllegalArgumentException(
-              "cannot change vector dimension from "
-                  + props.numDimensions
-                  + " to "
-                  + numDimensions
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-        if (props.searchStrategy != searchStrategy) {
-          throw new IllegalArgumentException(
-              "cannot change vector search strategy from "
-                  + props.searchStrategy
-                  + " to "
-                  + searchStrategy
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-      }
+    /**
+     * Returns true if the {@code fieldName} exists in the map and is of the same {@code dvType},
+     * and it is docValues only field.
+     */
+    synchronized boolean containsDvOnlyField(String fieldName, DocValuesType dvType) {
+      if (nameToNumber.containsKey(fieldName) == false) return false;
+      if (dvType != docValuesType.get(fieldName)) return false;
+      FieldDimensions fdimensions = dimensions.get(fieldName);
+      if (fdimensions != null && fdimensions.dimensionCount != 0) return false;
+      IndexOptions ioptions = indexOptions.get(fieldName);
+      if (ioptions != null && ioptions != IndexOptions.NONE) return false;
+      FieldVectorProperties fvp = vectorProps.get(fieldName);
+      if (fvp != null && fvp.numDimensions != 0) return false;
+      return true;
     }
 
     /**
-     * Returns true if the {@code fieldName} exists in the map and is of the same {@code dvType}.
+     * Construct a new FieldInfo based on the options in global field numbers This method is not
+     * synchronized as all the options it uses are not modifiable.

Review comment:
       From the point of view `FieldInfos.FieldNumbers`,  doc values type is not modifiable and set only once on adding this field to global field numbers.
   
   This method `constructFieldInfo` also checks the the provided in arguments `dvType` matches the one stored in the global field numbers for this field.  So I am not sure how it can be modifiable.
   
   Please let me know if I am missing something.




----------------------------------------------------------------
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-solr] mayya-sharipova commented on a change in pull request #2186: LUCENE-9334 Consistency of field data structures

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



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -455,8 +373,42 @@ synchronized int addOrGet(
           fieldNumber = lowestUnassignedFieldNumber;
         }
         assert fieldNumber >= 0;
+        FieldInfo.checkOptionsCorrectness(
+            fieldName,
+            false,

Review comment:
       Addressed [here](https://github.com/apache/lucene/pull/11/commits/0fe3493110ac2a5f750ad41f732436daff6c69f5#diff-b81932cf39b70347481b9659dd55b9b373427c2992cdacdfbc10ff8f4ea9eec0L378)




----------------------------------------------------------------
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-solr] mayya-sharipova edited a comment on pull request #2186: LUCENE-9334 Consistency of field data structures

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


   @jpountz  Adrien, I also wanted to ask your opinion about the behaviour of `writer.updateDocValues(Term term, fieldX)`:
   1.  What should happen if `fieldX` doesn't exist in the segment (but exists globally)? 
   2. What should happen if `fieldX` doesn't have docValues enabled?
   
   Before in both of these cases `updateDocValues` worked well creating a field if it doesn't exist or enabling doc values.
   
   But with current PR as the field schema is fixed, we can:
   1. Allow the 1st case  ?
   2. Throw an error on 2nd case, as docValues are not enabled.
   
   WDYT?


----------------------------------------------------------------
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-solr] mayya-sharipova commented on a change in pull request #2186: LUCENE-9334 Consistency of field data structures

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



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -130,127 +167,255 @@ 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,
+  void verifySameSchema(
+      IndexOptions indexOptions,
       boolean omitNorms,
       boolean storePayloads,
-      IndexOptions indexOptions,
-      Map<String, String> attributes,
+      boolean storeTermVector,
+      DocValuesType docValuesType,
+      long dvGen,
       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);
-      }
+      int dimensionNumBytes,
+      int vectorDimension,
+      VectorValues.SearchStrategy searchStrategy) {
+    verifySameIndexOptions(this.name, this.indexOptions, indexOptions);
+    if (this.indexOptions != IndexOptions.NONE) {
+      verifySamePostingsOptions(
+          this.name,
+          this.storeTermVector,
+          this.omitNorms,
+          this.storePayloads,
+          storeTermVector,
+          omitNorms,
+          storePayloads);
+    }
+    verifySameDocValuesType(this.name, this.docValuesType, docValuesType);
+    verifySameDVGen(this.name, this.dvGen, dvGen);
+    verifySamePointsOptions(
+        this.name,
+        this.pointDimensionCount,
+        this.pointIndexDimensionCount,
+        this.pointNumBytes,
+        dimensionCount,
+        indexDimensionCount,
+        dimensionNumBytes);
+    verifySameVectorOptions(
+        this.name,
+        this.vectorDimension,
+        this.vectorSearchStrategy,
+        vectorDimension,
+        searchStrategy);
+  }
+
+  /**
+   * Very that the provided index options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameIndexOptions(String fieldName, IndexOptions io1, IndexOptions io2) {
+    if (io1 != io2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from index options="
+              + io1
+              + " to inconsistent index options="
+              + io2);
     }
+  }
 
-    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)) {
+  /**
+   * Very that the provided docValues type are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameDocValuesType(
+      String fieldName, DocValuesType dv1, DocValuesType dv2) {
+    if (dv1 != dv2) {
       throw new IllegalArgumentException(
           "cannot change field \""
-              + name
+              + fieldName
+              + "\" from doc values type="
+              + dv1
+              + " to inconsistent doc values type="
+              + dv2);
+    }
+  }
+
+  /**
+   * Very that the provided doc values generations are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameDVGen(String fieldName, long dvg1, long dvg2) {
+    if (dvg1 != dvg2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from doc values generation="
+              + dvg1
+              + " to inconsistent doc values generation="
+              + dvg2);
+    }
+  }
+
+  /**
+   * Very that the provided posting options are the same

Review comment:
       Addressed in 6e7540ebd0ef79536cffabcf0ddc7a592b792252




----------------------------------------------------------------
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-solr] mayya-sharipova closed pull request #2186: LUCENE-9334 Consistency of field data structures

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


   


----------------------------------------------------------------
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-solr] mayya-sharipova commented on pull request #2186: LUCENE-9334 Consistency of field data structures

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


   @jpountz  Hi Adrien. Thanks for your review of the PR: https://github.com/apache/lucene-solr/pull/2186. I will go through your review and address your comments. I wanted to ask two assumptions I have, and check if we are ok with them: 
   
   1) A first doc with a field introduces FieldInfo for this field for the whole index, even if eventually this doc doesn't get indexed (e.g. if this doc has a too big stored field, the whole doc will be rolled back and deleted).  
   
   2) Doc values updates (`IndexWriter#updateDocValues`) are only applicable for fields that are indexed with doc values only. This was not the case before, and for example we could update doc values for a field that was indexed with postings. 
   


----------------------------------------------------------------
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-solr] mayya-sharipova commented on a change in pull request #2186: LUCENE-9334 Consistency of field data structures

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



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -481,205 +433,102 @@ synchronized int addOrGet(
                 + fieldName
                 + "] as non-soft-deletes already");
       }
-
       return fieldNumber.intValue();
     }
 
-    synchronized void verifyConsistent(Integer number, String name, IndexOptions indexOptions) {
-      if (name.equals(numberToName.get(number)) == false) {
-        throw new IllegalArgumentException(
-            "field number "
-                + number
-                + " is already mapped to field name \""
-                + numberToName.get(number)
-                + "\", not \""
-                + name
-                + "\"");
-      }
-      if (number.equals(nameToNumber.get(name)) == false) {
-        throw new IllegalArgumentException(
-            "field name \""
-                + name
-                + "\" is already mapped to field number \""
-                + nameToNumber.get(name)
-                + "\", not \""
-                + number
-                + "\"");
-      }
-      IndexOptions currentIndexOptions = this.indexOptions.get(name);
-      if (indexOptions != IndexOptions.NONE
-          && currentIndexOptions != null
-          && currentIndexOptions != IndexOptions.NONE
-          && indexOptions != currentIndexOptions) {
-        throw new IllegalArgumentException(
-            "cannot change field \""
-                + name
-                + "\" from index options="
-                + currentIndexOptions
-                + " to inconsistent index options="
-                + indexOptions);
-      }
-    }
-
-    synchronized void verifyConsistent(Integer number, String name, DocValuesType dvType) {
-      if (name.equals(numberToName.get(number)) == false) {
-        throw new IllegalArgumentException(
-            "field number "
-                + number
-                + " is already mapped to field name \""
-                + numberToName.get(number)
-                + "\", not \""
-                + name
-                + "\"");
-      }
-      if (number.equals(nameToNumber.get(name)) == false) {
-        throw new IllegalArgumentException(
-            "field name \""
-                + name
-                + "\" is already mapped to field number \""
-                + nameToNumber.get(name)
-                + "\", not \""
-                + number
-                + "\"");
-      }
-      DocValuesType currentDVType = docValuesType.get(name);
-      if (dvType != DocValuesType.NONE
-          && currentDVType != null
-          && currentDVType != DocValuesType.NONE
-          && dvType != currentDVType) {
-        throw new IllegalArgumentException(
-            "cannot change DocValues type from "
-                + currentDVType
-                + " to "
-                + dvType
-                + " for field \""
-                + name
-                + "\"");
-      }
-    }
-
-    synchronized void verifyConsistentDimensions(
-        Integer number,
-        String name,
-        int dataDimensionCount,
+    private void verifySameSchema(
+        String fieldName,
+        IndexOptions indexOptions,
+        boolean storeTermVector,
+        boolean omitNorms,
+        DocValuesType dvType,
+        int dimensionCount,
         int indexDimensionCount,
-        int dimensionNumBytes) {
-      if (name.equals(numberToName.get(number)) == false) {
-        throw new IllegalArgumentException(
-            "field number "
-                + number
-                + " is already mapped to field name \""
-                + numberToName.get(number)
-                + "\", not \""
-                + name
-                + "\"");
-      }
-      if (number.equals(nameToNumber.get(name)) == false) {
-        throw new IllegalArgumentException(
-            "field name \""
-                + name
-                + "\" is already mapped to field number \""
-                + nameToNumber.get(name)
-                + "\", not \""
-                + number
-                + "\"");
-      }
-      FieldDimensions dim = dimensions.get(name);
-      if (dim != null) {
-        if (dim.dimensionCount != dataDimensionCount) {
-          throw new IllegalArgumentException(
-              "cannot change point dimension count from "
-                  + dim.dimensionCount
-                  + " to "
-                  + dataDimensionCount
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-        if (dim.indexDimensionCount != indexDimensionCount) {
-          throw new IllegalArgumentException(
-              "cannot change point index dimension count from "
-                  + dim.indexDimensionCount
-                  + " to "
-                  + indexDimensionCount
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-        if (dim.dimensionNumBytes != dimensionNumBytes) {
-          throw new IllegalArgumentException(
-              "cannot change point numBytes from "
-                  + dim.dimensionNumBytes
-                  + " to "
-                  + dimensionNumBytes
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-      }
+        int dimensionNumBytes,
+        int vectorDimension,
+        VectorValues.SearchStrategy searchStrategy) {
+
+      IndexOptions currentOpts = this.indexOptions.get(fieldName);
+      verifySameIndexOptions(fieldName, currentOpts, indexOptions);
+      if (currentOpts != IndexOptions.NONE) {
+        boolean curStoreTermVector = this.storeTermVectors.get(fieldName);
+        verifySameStoreTermVectors(fieldName, curStoreTermVector, storeTermVector);
+        boolean curOmitNorms = this.omitNorms.get(fieldName);
+        verifySameOmitNorms(fieldName, curOmitNorms, omitNorms);
+      }
+
+      DocValuesType currentDVType = docValuesType.get(fieldName);
+      verifySameDocValuesType(fieldName, currentDVType, dvType);
+
+      FieldDimensions dims = dimensions.get(fieldName);
+      verifySamePointsOptions(
+          fieldName,
+          dims.dimensionCount,
+          dims.indexDimensionCount,
+          dims.dimensionNumBytes,
+          dimensionCount,
+          indexDimensionCount,
+          dimensionNumBytes);
+
+      FieldVectorProperties props = vectorProps.get(fieldName);
+      verifySameVectorOptions(
+          fieldName, props.numDimensions, props.searchStrategy, vectorDimension, searchStrategy);
     }
 
-    synchronized void verifyConsistentVectorProperties(
-        Integer number,
-        String name,
-        int numDimensions,
-        VectorValues.SearchStrategy searchStrategy) {
-      if (name.equals(numberToName.get(number)) == false) {
-        throw new IllegalArgumentException(
-            "field number "
-                + number
-                + " is already mapped to field name \""
-                + numberToName.get(number)
-                + "\", not \""
-                + name
-                + "\"");
-      }
-      if (number.equals(nameToNumber.get(name)) == false) {
-        throw new IllegalArgumentException(
-            "field name \""
-                + name
-                + "\" is already mapped to field number \""
-                + nameToNumber.get(name)
-                + "\", not \""
-                + number
-                + "\"");
-      }
-      FieldVectorProperties props = vectorProps.get(name);
-      if (props != null) {
-        if (props.numDimensions != numDimensions) {
-          throw new IllegalArgumentException(
-              "cannot change vector dimension from "
-                  + props.numDimensions
-                  + " to "
-                  + numDimensions
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-        if (props.searchStrategy != searchStrategy) {
-          throw new IllegalArgumentException(
-              "cannot change vector search strategy from "
-                  + props.searchStrategy
-                  + " to "
-                  + searchStrategy
-                  + " for field=\""
-                  + name
-                  + "\"");
-        }
-      }
+    /**
+     * Returns true if the {@code fieldName} exists in the map and is of the same {@code dvType},
+     * and it is docValues only field.
+     */
+    synchronized boolean containsDvOnlyField(String fieldName, DocValuesType dvType) {
+      if (nameToNumber.containsKey(fieldName) == false) return false;
+      if (dvType != docValuesType.get(fieldName)) return false;
+      FieldDimensions fdimensions = dimensions.get(fieldName);
+      if (fdimensions != null && fdimensions.dimensionCount != 0) return false;
+      IndexOptions ioptions = indexOptions.get(fieldName);
+      if (ioptions != null && ioptions != IndexOptions.NONE) return false;
+      FieldVectorProperties fvp = vectorProps.get(fieldName);
+      if (fvp != null && fvp.numDimensions != 0) return false;
+      return true;
     }
 
     /**
-     * Returns true if the {@code fieldName} exists in the map and is of the same {@code dvType}.
+     * Construct a new FieldInfo based on the options in global field numbers This method is not
+     * synchronized as all the options it uses are not modifiable.
+     *
+     * @param fieldName name of the field
+     * @param dvType doc values type
+     * @param newFieldNumber a new field number
+     * @return {@code null} if {@code fieldName} doesn't exist in the map or is not of the same
+     *     {@code dvType} returns a new FieldInfo based based on the options in global field numbers
      */
-    synchronized boolean contains(String fieldName, DocValuesType dvType) {
-      // used by IndexWriter.updateNumericDocValue
-      if (!nameToNumber.containsKey(fieldName)) {
-        return false;
-      } else {
-        // only return true if the field has the same dvType as the requested one
-        return dvType == docValuesType.get(fieldName);
-      }
+    FieldInfo constructFieldInfo(String fieldName, DocValuesType dvType, int newFieldNumber) {
+      Integer fieldNumber;
+      synchronized (this) {
+        fieldNumber = nameToNumber.get(fieldName);
+      }
+      if (fieldNumber == null) return null;
+      DocValuesType dvType0 = docValuesType.get(fieldName);
+      if (dvType != dvType0) return null;
+
+      boolean isSoftDeletesField = fieldName.equals(softDeletesFieldName);
+      FieldDimensions dims = dimensions.get(fieldName);
+      FieldVectorProperties vectors = vectorProps.get(fieldName);

Review comment:
       Addressed [here](https://github.com/apache/lucene/pull/11/commits/0fe3493110ac2a5f750ad41f732436daff6c69f5#diff-b81932cf39b70347481b9659dd55b9b373427c2992cdacdfbc10ff8f4ea9eec0L504)




----------------------------------------------------------------
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-solr] jpountz commented on a change in pull request #2186: LUCENE-9334 Consistency of field data structures

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



##########
File path: lucene/test-framework/src/java/org/apache/lucene/index/BaseVectorFormatTestCase.java
##########
@@ -131,7 +115,8 @@ public void testIllegalDimChangeTwoDocs() throws Exception {
       Document doc = new Document();
       doc.add(new VectorField("f", new float[4], VectorValues.SearchStrategy.DOT_PRODUCT_HNSW));
       w.addDocument(doc);
-      if (random().nextBoolean()) {
+      boolean rb = random().nextBoolean();
+      if (rb) {

Review comment:
       nit: maybe we should make sure we test both branches all the time to have good coverage, especially now that the error messages differ? (and likewise for other places that have similar logic)

##########
File path: lucene/test-framework/src/java/org/apache/lucene/index/BasePointsFormatTestCase.java
##########
@@ -1174,8 +1174,11 @@ public void testMixedSchema() throws Exception {
 
     Document doc = new Document();
     doc.add(new IntPoint("id", 0));
-    w.addDocument(doc);
-    // now we write another segment where the id field does have points:
+    IllegalArgumentException ex =
+        expectThrows(IllegalArgumentException.class, () -> w.addDocument(doc));
+    assertEquals(
+        "cannot change field \"id\" from index options=DOCS to inconsistent index options=NONE",
+        ex.getMessage());

Review comment:
       I think we should refactor or drop this test, as it is not testing the points format now, but IndexingChain/FieldsInfos' logic. Maybe we could rename the test `testMergeMissing` and configure the first segment to not have the `id` field 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-solr] mayya-sharipova commented on a change in pull request #2186: LUCENE-9334 Consistency of field data structures

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



##########
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) {

Review comment:
       addressed [here](https://github.com/apache/lucene/pull/11/commits/0fe3493110ac2a5f750ad41f732436daff6c69f5#diff-b9ceb6f65b3597b6c2406c2d41cc7db39e04e21bb77a13fcc92eb0429c444d46L244)




----------------------------------------------------------------
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-solr] jpountz commented on pull request #2186: LUCENE-9334 Consistency of field data structures

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


   @mayya-sharipova Yes, if the field has been added previously without doc values, then it should be illegal to add it later with doc values indeed.


----------------------------------------------------------------
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-solr] mayya-sharipova commented on pull request #2186: LUCENE-9334 Consistency of field data structures

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


   @jpountz  Adrien, thank a lot for the initial review. Sorry for the delay, the patch turned out to be much more involved than I expected.
   There are still few tests failing, but I will try to fix them soon.
   
   
   >  Can you give a bit more details about how this PR works at a high level? E.g. how does it handle the case when two threads are concurrently trying to add a field with different schemas?
   
   ---
   **Segment level**: ensuring a field has the same schema across all the documents of the segment.
   - We use `FieldSchema` for this, that's an internal field of `PerField`. It represents a schema of the field in the current document. 
   - With every new document we reset `FieldSchema`.  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 `FieldSchema` of the current document with the corresponding `FieldInfo `(FieldInfo is built on a first document in which we encounter this field).
   
   Relevant code in `IndexingChaing:processDocument`:
   ```java
   ...
          if (pf.fieldGen != fieldGen) { // first time we see this field in this document
             ....
             pf.reset(docID);
           }
   ...
   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);
     }
   }
   ```
   
   ---
   **Index level**: ensuring a field has the same schema across all the documents of the index.
   
   - This check is done in `FieldInfos`. 
   - When we encounter a new field in a segment, we try to initialize it in `IndexingChain::initializeFieldInfo` with  the options from the current `FieldSchema`.  This calls `FieldInfos.Builder::add` ->  `FieldInfos.Builder::addField`.  
   -  The  first thing in `FieldInfos.Builder::addField`  is to call `globalFieldNumbers.addOrGet` with the given schema options. `globalFieldNumbers.addOrGet`  is a synchronized method, and as I understood  `FieldNumbers` are shared across all indexing threads of the same index.  
   - `globalFieldNumbers.addOrGet` for a field it sees the first time, will initialize all its maps `indexOptions`, `docValuesType` etc.  If the field already exists, it will check that the given in parameters schema options are the same as stored in its maps for the given field.
   - As `globalFieldNumbers.addOrGet` is a synchronized method only a single thread will be able to initialize schema options for a field. All other threads that deal with the same field, must confirm with the same field schema.
   
   @jpountz  Is my assumption and logic correct here? 
   
   
   


----------------------------------------------------------------
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-solr] mayya-sharipova edited a comment on pull request #2186: LUCENE-9334 Consistency of field data structures

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


   @jpountz  Adrien, I also wanted to ask your opinion about the behaviour of `writer.updateDocValues(Term term, fieldX)`, as I am not sure what to do in this case:
   1.  What should happen if `fieldX` doesn't exist in the segment (but exists globally)? 
   2. What should happen if `fieldX` doesn't have docValues enabled?
   
   Before in both of these cases `updateDocValues` worked well creating a field if it doesn't exist or enabling doc values.
   
   But with current PR as the field schema is fixed, we can:
   1. Allow the 1st case  ?
   2. Throw an error on 2nd case, as docValues are not enabled.
   
   WDYT?


----------------------------------------------------------------
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-solr] mayya-sharipova commented on a change in pull request #2186: LUCENE-9334 Consistency of field data structures

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



##########
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:
       Addressed [here](https://github.com/apache/lucene/pull/11/commits/0fe3493110ac2a5f750ad41f732436daff6c69f5#diff-b81932cf39b70347481b9659dd55b9b373427c2992cdacdfbc10ff8f4ea9eec0L674)




----------------------------------------------------------------
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-solr] mayya-sharipova edited a comment on pull request #2186: LUCENE-9334 Consistency of field data structures

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


   @jpountz  Adrien, I also wanted to ask your opinion about the behaviour of `writer.updateDocValues(Term term, fieldX)`:
   1.  What should happen if `fieldX` doesn't exist in the segment (but exists globally)? 
   2. What should happen if `fieldX` doesn't have docValues enabled?
   
   Before in both of these cases `updateDocValues` worked well creating a field if it doesn't exist or enabling doc values.
   
   But with current PR as the field schema is fixed, we can:
   1. Allow the 1st case  with the assumption that the created field will only has docValues and nothing more.
   2. Throw an error on 2nd case, as docValues are not enabled.
   
   WDYT?


----------------------------------------------------------------
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-solr] mayya-sharipova commented on a change in pull request #2186: LUCENE-9334 Consistency of field data structures

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



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexingChain.java
##########
@@ -349,7 +350,8 @@ private void writePoints(SegmentWriteState state, Sorter.DocMap sortMap) throws
 
             perField.pointValuesWriter.flush(state, sortMap, pointsWriter);
             perField.pointValuesWriter = null;
-          } else if (perField.fieldInfo.getPointDimensionCount() != 0) {
+          } else if (perField.fieldInfo != null

Review comment:
       Not, always.
   We first create PerField: `PerField pf = getOrAddPerField(field.name(), fieldType);` and then initialize its FieldInfo `initializeFieldInfo(pf)`. Initialization of FieldInfo may fail, if there is already a global field with the same name but a different schema. So we may end up with `PerField` with a null `fieldInfo`.




----------------------------------------------------------------
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-solr] mayya-sharipova commented on a change in pull request #2186: LUCENE-9334 Consistency of field data structures

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



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -1860,8 +1862,8 @@ public long softUpdateDocument(
    */
   public long updateNumericDocValue(Term term, String field, long value) throws IOException {
     ensureOpen();
-    if (!globalFieldNumberMap.contains(field, DocValuesType.NUMERIC)) {
-      throw new IllegalArgumentException("can only update existing numeric-docvalues fields!");
+    if (globalFieldNumberMap.containsDvOnlyField(field, DocValuesType.NUMERIC) == false) {
+      throw new IllegalArgumentException("can only update existing numeric docvalues only fields!");
     }

Review comment:
       Addressed [here](https://github.com/apache/lucene/pull/11/commits/0fe3493110ac2a5f750ad41f732436daff6c69f5#diff-b81932cf39b70347481b9659dd55b9b373427c2992cdacdfbc10ff8f4ea9eec0L482)




----------------------------------------------------------------
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-solr] mayya-sharipova edited a comment on pull request #2186: LUCENE-9334 Consistency of field data structures

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


   @jpountz  Adrien, I also wanted to ask your opinion about the behaviour of `writer.updateDocValues(Term term, fieldX)`:
   1.  What should happen if `fieldX` doesn't exist in the segment (but exists globally)? 
   2. What should happen if `fieldX` doesn't have docValues enabled in the segment?
   
   Before in both of these cases `updateDocValues` worked well creating a field if it doesn't exist or enabling doc values.
   
   But with current PR as the field schema is fixed, we can:
   1. Allow the 1st case  with the assumption that the created field will only has docValues and nothing more.
   2. Throw an error on 2nd case, as docValues are not enabled.
   
   WDYT?


----------------------------------------------------------------
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-solr] mayya-sharipova commented on a change in pull request #2186: LUCENE-9334 Consistency of field data structures

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



##########
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(

Review comment:
       addressed [here](https://github.com/apache/lucene/pull/11/commits/0fe3493110ac2a5f750ad41f732436daff6c69f5#diff-b9ceb6f65b3597b6c2406c2d41cc7db39e04e21bb77a13fcc92eb0429c444d46L135)




----------------------------------------------------------------
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-solr] jpountz commented on pull request #2186: LUCENE-9334 Consistency of field data structures

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


   @mayya-sharipova These assumptions sound right to me.


----------------------------------------------------------------
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-solr] mayya-sharipova commented on pull request #2186: LUCENE-9334 Consistency of field data structures

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


   @jpountz  Thanks for your feedback. I have [moved](https://github.com/apache/lucene/pull/11)  the PR to the new Lucene repo and will address your feedback there.
   
   Closing this 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-solr] mayya-sharipova commented on a change in pull request #2186: LUCENE-9334 Consistency of field data structures

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



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexingChain.java
##########
@@ -1313,4 +1259,110 @@ public void recycleIntBlocks(int[][] blocks, int offset, int length) {
       bytesUsed.addAndGet(-(length * (IntBlockPool.INT_BLOCK_SIZE * Integer.BYTES)));
     }
   }
+
+  private static final class FieldSchema {

Review comment:
       Addressed in 6e7540ebd0ef79536cffabcf0ddc7a592b792252




----------------------------------------------------------------
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-solr] jpountz commented on a change in pull request #2186: LUCENE-9334 Consistency of field data structures

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



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -130,127 +167,255 @@ 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,
+  void verifySameSchema(
+      IndexOptions indexOptions,
       boolean omitNorms,
       boolean storePayloads,
-      IndexOptions indexOptions,
-      Map<String, String> attributes,
+      boolean storeTermVector,
+      DocValuesType docValuesType,
+      long dvGen,
       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);
-      }
+      int dimensionNumBytes,
+      int vectorDimension,
+      VectorValues.SearchStrategy searchStrategy) {
+    verifySameIndexOptions(this.name, this.indexOptions, indexOptions);
+    if (this.indexOptions != IndexOptions.NONE) {
+      verifySamePostingsOptions(
+          this.name,
+          this.storeTermVector,
+          this.omitNorms,
+          this.storePayloads,
+          storeTermVector,
+          omitNorms,
+          storePayloads);
+    }
+    verifySameDocValuesType(this.name, this.docValuesType, docValuesType);
+    verifySameDVGen(this.name, this.dvGen, dvGen);
+    verifySamePointsOptions(
+        this.name,
+        this.pointDimensionCount,
+        this.pointIndexDimensionCount,
+        this.pointNumBytes,
+        dimensionCount,
+        indexDimensionCount,
+        dimensionNumBytes);
+    verifySameVectorOptions(
+        this.name,
+        this.vectorDimension,
+        this.vectorSearchStrategy,
+        vectorDimension,
+        searchStrategy);
+  }
+
+  /**
+   * Very that the provided index options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameIndexOptions(String fieldName, IndexOptions io1, IndexOptions io2) {
+    if (io1 != io2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from index options="
+              + io1
+              + " to inconsistent index options="
+              + io2);
     }
+  }
 
-    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)) {
+  /**
+   * Very that the provided docValues type are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameDocValuesType(
+      String fieldName, DocValuesType dv1, DocValuesType dv2) {
+    if (dv1 != dv2) {
       throw new IllegalArgumentException(
           "cannot change field \""
-              + name
+              + fieldName
+              + "\" from doc values type="
+              + dv1
+              + " to inconsistent doc values type="
+              + dv2);
+    }
+  }
+
+  /**
+   * Very that the provided doc values generations are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameDVGen(String fieldName, long dvg1, long dvg2) {
+    if (dvg1 != dvg2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from doc values generation="
+              + dvg1
+              + " to inconsistent doc values generation="
+              + dvg2);
+    }
+  }
+
+  /**
+   * Very that the provided posting options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySamePostingsOptions(
+      String fieldName,
+      boolean stv1,
+      boolean on1,
+      boolean sp1,
+      boolean stv2,
+      boolean on2,
+      boolean sp2) {
+    if (stv1 != stv2 || on1 != on2 || sp1 != sp2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from storeTermVector="
+              + stv1
+              + ", omitNorms="
+              + on1
+              + ", storePayloads="
+              + sp1
+              + " to inconsistent storeTermVector="
+              + stv2
+              + ", omitNorms="
+              + on2
+              + ", storePayloads="
+              + sp2);
+    }
+  }
+
+  /**
+   * Very that the provided points indexing options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySamePointsOptions(
+      String fieldName, int dc1, int idc1, int nb1, int dc2, int idc2, int nb2) {
+    if (dc1 != dc2 || idc1 != idc2 || nb1 != nb2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
               + "\" from points dimensionCount="
-              + this.pointDimensionCount
+              + dc1

Review comment:
       I liked the descriptive variable names better.

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexingChain.java
##########
@@ -1313,4 +1259,110 @@ public void recycleIntBlocks(int[][] blocks, int offset, int length) {
       bytesUsed.addAndGet(-(length * (IntBlockPool.INT_BLOCK_SIZE * Integer.BYTES)));
     }
   }
+
+  private static final class FieldSchema {

Review comment:
       can you add a bit more docs, e.g. how this it differ from FieldInfo?

##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
##########
@@ -130,127 +167,255 @@ 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,
+  void verifySameSchema(
+      IndexOptions indexOptions,
       boolean omitNorms,
       boolean storePayloads,
-      IndexOptions indexOptions,
-      Map<String, String> attributes,
+      boolean storeTermVector,
+      DocValuesType docValuesType,
+      long dvGen,
       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);
-      }
+      int dimensionNumBytes,
+      int vectorDimension,
+      VectorValues.SearchStrategy searchStrategy) {
+    verifySameIndexOptions(this.name, this.indexOptions, indexOptions);
+    if (this.indexOptions != IndexOptions.NONE) {
+      verifySamePostingsOptions(
+          this.name,
+          this.storeTermVector,
+          this.omitNorms,
+          this.storePayloads,
+          storeTermVector,
+          omitNorms,
+          storePayloads);
+    }
+    verifySameDocValuesType(this.name, this.docValuesType, docValuesType);
+    verifySameDVGen(this.name, this.dvGen, dvGen);
+    verifySamePointsOptions(
+        this.name,
+        this.pointDimensionCount,
+        this.pointIndexDimensionCount,
+        this.pointNumBytes,
+        dimensionCount,
+        indexDimensionCount,
+        dimensionNumBytes);
+    verifySameVectorOptions(
+        this.name,
+        this.vectorDimension,
+        this.vectorSearchStrategy,
+        vectorDimension,
+        searchStrategy);
+  }
+
+  /**
+   * Very that the provided index options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameIndexOptions(String fieldName, IndexOptions io1, IndexOptions io2) {
+    if (io1 != io2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from index options="
+              + io1
+              + " to inconsistent index options="
+              + io2);
     }
+  }
 
-    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)) {
+  /**
+   * Very that the provided docValues type are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameDocValuesType(
+      String fieldName, DocValuesType dv1, DocValuesType dv2) {
+    if (dv1 != dv2) {
       throw new IllegalArgumentException(
           "cannot change field \""
-              + name
+              + fieldName
+              + "\" from doc values type="
+              + dv1
+              + " to inconsistent doc values type="
+              + dv2);
+    }
+  }
+
+  /**
+   * Very that the provided doc values generations are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  public static void verifySameDVGen(String fieldName, long dvg1, long dvg2) {
+    if (dvg1 != dvg2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from doc values generation="
+              + dvg1
+              + " to inconsistent doc values generation="
+              + dvg2);
+    }
+  }
+
+  /**
+   * Very that the provided posting options are the same

Review comment:
       Several javadocs seem to have this typo.
   
   ```suggestion
      * Verify that the provided posting options are the same
   ```




----------------------------------------------------------------
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-solr] mayya-sharipova commented on pull request #2186: LUCENE-9334 Consistency of field data structures

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


   @jpountz  Adrien, I also wanted to ask your opinion about the behaviour of `writer.updateDocValues(Term term, fieldX)`:
   1.  What should happen if `fieldX` doesn't exist? 
   2. What should happen if `fieldX` doesn't have docValues enabled?
   
   Before in both of these cases `updateDocValues` worked well creating a field if it doesn't exist or enabling doc values.
   
   But with current PR as the field schema is fixed, we can:
   1. Allow the 1st case  with the assumption that the created field will only has docValues and nothing more.
   2. Throw an error on 2nd case, as docValues are not enabled.
   
   WDYT?


----------------------------------------------------------------
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-solr] mayya-sharipova edited a comment on pull request #2186: LUCENE-9334 Consistency of field data structures

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


   @jpountz  Adrien, I also wanted to ask your opinion about the behaviour of `writer.updateDocValues(Term term, fieldX)`, as I am not sure what to do in this case:
   1.  What should happen if `fieldX` doesn't exist in the segment (but exists globally)? 
   2. What should happen if `fieldX` doesn't have docValues enabled?
   
   Before in both of these cases `updateDocValues` worked well creating a new `FieldInfo` in the current segment if it doesn't exist or enabling doc values.
   
   But with current PR as the field schema is fixed, we can:
   1. Thrown an  error? Or allow only in a case where the global `fieldX`  is only a docValues field?
   2. Throw an error, as docValues are not enabled and this can't be changed.
   
   WDYT?


----------------------------------------------------------------
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