You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2021/04/14 19:00:49 UTC

[lucene] branch main updated: LUCENE-9334 Consistency of field data structures

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

mayya pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/lucene.git


The following commit(s) were added to refs/heads/main by this push:
     new d03662c  LUCENE-9334 Consistency of field data structures
d03662c is described below

commit d03662c48bfc5bf2be4840a7f743f9cb64b17fee
Author: Mayya Sharipova <ma...@elastic.co>
AuthorDate: Wed Apr 14 15:00:41 2021 -0400

    LUCENE-9334 Consistency of field data structures
    
    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. Thus, for example, it is not allowed to have one document
    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.
    
    As a consequence of this, doc values updates are
    only applicable for fields that are indexed with doc values only.
---
 lucene/CHANGES.txt                                 |   7 +
 lucene/MIGRATE.md                                  |  19 +
 .../java/org/apache/lucene/index/FieldInfo.java    | 393 +++++-----
 .../java/org/apache/lucene/index/FieldInfos.java   | 791 +++++++--------------
 .../java/org/apache/lucene/index/IndexWriter.java  |  61 +-
 .../org/apache/lucene/index/IndexingChain.java     | 618 +++++++++-------
 .../org/apache/lucene/index/PointValuesWriter.java |   9 +
 .../org/apache/lucene/index/ReadersAndUpdates.java |  11 +-
 .../apache/lucene/index/TermVectorsConsumer.java   |   6 +-
 .../lucene/index/TermVectorsConsumerPerField.java  |   3 -
 .../lucene/document/TestPerFieldConsistency.java   | 224 ++++++
 .../org/apache/lucene/index/TestAtomicUpdate.java  |   6 +-
 .../lucene/index/TestBinaryDocValuesUpdates.java   |  57 +-
 .../test/org/apache/lucene/index/TestCodecs.java   |  30 +-
 .../lucene/index/TestConsistentFieldNumbers.java   |   8 +-
 .../apache/lucene/index/TestDocValuesIndexing.java |  23 -
 .../org/apache/lucene/index/TestFieldInfos.java    | 128 +++-
 .../org/apache/lucene/index/TestFieldsReader.java  |  26 +-
 .../org/apache/lucene/index/TestIndexOptions.java  |  35 +-
 .../org/apache/lucene/index/TestIndexSorting.java  |   4 +-
 .../org/apache/lucene/index/TestIndexWriter.java   |  14 +-
 .../lucene/index/TestIndexWriterDeleteByQuery.java |  70 --
 .../apache/lucene/index/TestIndexWriterReader.java |   2 +-
 .../lucene/index/TestMixedDocValuesUpdates.java    |  37 +-
 .../lucene/index/TestNumericDocValuesUpdates.java  |  33 +-
 .../org/apache/lucene/index/TestOmitNorms.java     | 200 +-----
 .../org/apache/lucene/index/TestPointValues.java   |  66 +-
 .../apache/lucene/index/TestPostingsOffsets.java   |  35 -
 .../lucene/index/TestSortingCodecReader.java       |  14 +-
 .../apache/lucene/index/TestStressIndexing2.java   | 122 ++--
 .../apache/lucene/index/TestTermVectorsWriter.java |  23 +-
 .../directory/TestBackwardsCompatibility.java      |   5 +
 .../grouping/TestDistinctValuesCollector.java      |   2 +-
 .../lucene/search/grouping/TestGrouping.java       |   1 +
 .../apache/lucene/search/join/TestBlockJoin.java   |   1 +
 .../lucene/search/join/TestBlockJoinScorer.java    |   2 +-
 .../apache/lucene/search/join/TestJoinUtil.java    |   1 +
 .../luke/models/overview/OverviewTestBase.java     |   4 +-
 .../search/suggest/TestDocumentDictionary.java     |   9 +-
 .../lucene/index/BaseFieldInfoFormatTestCase.java  | 106 +--
 .../lucene/index/BasePointsFormatTestCase.java     |   7 +-
 .../lucene/index/BaseVectorFormatTestCase.java     | 102 ++-
 .../java/org/apache/lucene/index/DocHelper.java    |  43 +-
 .../org/apache/lucene/util/LuceneTestCase.java     |  62 +-
 44 files changed, 1702 insertions(+), 1718 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 708fdd7..e981b3d 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -103,6 +103,13 @@ API Changes
 
 * LUCENE-9387: Removed CodecReader#ramBytesUsed. (Adrien Grand)
 
+* LUCENE-9334: Require consistency between data-structures on a per-field basis.
+  A field across all documents within an index must be indexed with the same index
+  options and data-structures. As a consequence of this, doc values updates are
+  only applicable for fields that are indexed with doc values only. (Mayya Sharipova,
+  Adrien Grand, Simon Willnauer)
+
+
 Improvements
 
 * LUCENE-9687: Hunspell support improvements: add API for spell-checking and suggestions, support compound words,
diff --git a/lucene/MIGRATE.md b/lucene/MIGRATE.md
index ecf0d2d..783efff 100644
--- a/lucene/MIGRATE.md
+++ b/lucene/MIGRATE.md
@@ -364,6 +364,25 @@ Sorting on a numeric field that is indexed with both doc values and points may u
 optimization to skip non-competitive documents. This optimization relies on the assumption
 that the same data is stored in these points and doc values.
 
+### Require consistency between data-structures on a per-field basis
+
+The per field data-structures are implicitly defined by the first document 
+indexed that contains a certain field. Once defined, the per field 
+data-structures are not changeable for the whole index. For example, if you 
+first index a document where a certain field is indexed with doc values and 
+points, all subsequent documents containing this field must also have this
+field indexed with only doc values and points.
+
+This also means that an index created in the previous version that doesn't 
+satisfy this requirement can not be updated. 
+
+### Doc values updates are allowed only for doc values only fields
+
+Previously IndexWriter could update doc values for a binary or numeric docValue 
+field that was also indexed with other data structures (e.g. postings, vectors 
+etc). This is not allowed anymore. A field must be indexed with only doc values 
+to be allowed for doc values updates in IndexWriter.
+
 ## SortedDocValues no longer extends BinaryDocValues (LUCENE-9796)
 
 SortedDocValues no longer extends BinaryDocValues: SortedDocValues do not have a per-document
diff --git a/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java b/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
index c77a0b8..bbad2bb 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
@@ -38,7 +38,7 @@ public final class FieldInfo {
 
   private boolean omitNorms; // omit norms associated with indexed fields
 
-  private IndexOptions indexOptions = IndexOptions.NONE;
+  private final IndexOptions indexOptions;
   private boolean storePayloads; // whether this field stores payloads together with term positions
 
   private final Map<String, String> attributes;
@@ -54,8 +54,9 @@ public final class FieldInfo {
   private int pointIndexDimensionCount;
   private int pointNumBytes;
 
-  private int vectorDimension; // if it is a positive value, it means this field indexes vectors
-  private VectorValues.SearchStrategy vectorSearchStrategy = VectorValues.SearchStrategy.NONE;
+  // if it is a positive value, it means this field indexes vectors
+  private final int vectorDimension;
+  private final VectorValues.SearchStrategy vectorSearchStrategy;
 
   // whether this field is used as the soft-deletes field
   private final boolean softDeletesField;
@@ -109,148 +110,267 @@ public final class FieldInfo {
     this.checkConsistency();
   }
 
-  /** Performs internal consistency checks. Always returns true (or throws IllegalStateException) */
-  public boolean checkConsistency() {
+  /**
+   * Check correctness of the FieldInfo options
+   *
+   * @throws IllegalArgumentException if some options are incorrect
+   */
+  public void checkConsistency() {
+    if (indexOptions == null) {
+      throw new IllegalArgumentException("IndexOptions must not be null (field: '" + name + "')");
+    }
     if (indexOptions != IndexOptions.NONE) {
       // Cannot store payloads unless positions are indexed:
       if (indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0 && storePayloads) {
-        throw new IllegalStateException(
+        throw new IllegalArgumentException(
             "indexed field '" + name + "' cannot have payloads without positions");
       }
     } else {
       if (storeTermVector) {
-        throw new IllegalStateException(
+        throw new IllegalArgumentException(
             "non-indexed field '" + name + "' cannot store term vectors");
       }
       if (storePayloads) {
-        throw new IllegalStateException("non-indexed field '" + name + "' cannot store payloads");
+        throw new IllegalArgumentException(
+            "non-indexed field '" + name + "' cannot store payloads");
       }
       if (omitNorms) {
-        throw new IllegalStateException("non-indexed field '" + name + "' cannot omit norms");
+        throw new IllegalArgumentException("non-indexed field '" + name + "' cannot omit norms");
       }
     }
 
-    if (pointDimensionCount < 0) {
-      throw new IllegalStateException(
-          "pointDimensionCount must be >= 0; got " + pointDimensionCount);
+    if (docValuesType == null) {
+      throw new IllegalArgumentException("DocValuesType must not be null (field: '" + name + "')");
+    }
+    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
+      throw new IllegalArgumentException(
+          "field '"
+              + name
+              + "' cannot have a docvalues update generation without having docvalues");
     }
 
+    if (pointDimensionCount < 0) {
+      throw new IllegalArgumentException(
+          "pointDimensionCount must be >= 0; got "
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
+    }
     if (pointIndexDimensionCount < 0) {
-      throw new IllegalStateException(
-          "pointIndexDimensionCount must be >= 0; got " + pointIndexDimensionCount);
+      throw new IllegalArgumentException(
+          "pointIndexDimensionCount must be >= 0; got "
+              + pointIndexDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes < 0) {
-      throw new IllegalStateException("pointNumBytes must be >= 0; got " + pointNumBytes);
+      throw new IllegalArgumentException(
+          "pointNumBytes must be >= 0; got " + pointNumBytes + " (field: '" + name + "')");
     }
 
     if (pointDimensionCount != 0 && pointNumBytes == 0) {
-      throw new IllegalStateException(
-          "pointNumBytes must be > 0 when pointDimensionCount=" + pointDimensionCount);
+      throw new IllegalArgumentException(
+          "pointNumBytes must be > 0 when pointDimensionCount="
+              + pointDimensionCount
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointIndexDimensionCount != 0 && pointDimensionCount == 0) {
-      throw new IllegalStateException(
-          "pointIndexDimensionCount must be 0 when pointDimensionCount=0");
+      throw new IllegalArgumentException(
+          "pointIndexDimensionCount must be 0 when pointDimensionCount=0"
+              + " (field: '"
+              + name
+              + "')");
     }
-
     if (pointNumBytes != 0 && pointDimensionCount == 0) {
-      throw new IllegalStateException(
-          "pointDimensionCount must be > 0 when pointNumBytes=" + pointNumBytes);
+      throw new IllegalArgumentException(
+          "pointDimensionCount must be > 0 when pointNumBytes="
+              + pointNumBytes
+              + " (field: '"
+              + name
+              + "')");
     }
 
-    if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
-      throw new IllegalStateException(
-          "field '"
+    if (vectorSearchStrategy == null) {
+      throw new IllegalArgumentException(
+          "Vector search strategy must not be null (field: '" + name + "')");
+    }
+    if (vectorDimension < 0) {
+      throw new IllegalArgumentException(
+          "vectorDimension must be >=0; got " + vectorDimension + " (field: '" + name + "')");
+    }
+    if (vectorDimension == 0 && vectorSearchStrategy != VectorValues.SearchStrategy.NONE) {
+      throw new IllegalArgumentException(
+          "vector search strategy must be NONE when dimension = 0; got "
+              + vectorSearchStrategy
+              + " (field: '"
               + name
-              + "' cannot have a docvalues update generation without having docvalues");
+              + "')");
     }
+  }
 
-    if (vectorDimension < 0) {
-      throw new IllegalStateException("vectorDimension must be >=0; got " + vectorDimension);
+  /**
+   * Verify that the provided FieldInfo has the same schema as this FieldInfo
+   *
+   * @param o – other FieldInfo whose schema is verified against this FieldInfo's schema
+   * @throws IllegalArgumentException if the field schemas are not the same
+   */
+  void verifySameSchema(FieldInfo o) {
+    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);
+    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
+   */
+  static void verifySameIndexOptions(
+      String fieldName, IndexOptions indexOptions1, IndexOptions indexOptions2) {
+    if (indexOptions1 != indexOptions2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from index options="
+              + indexOptions1
+              + " to inconsistent index options="
+              + indexOptions2);
     }
+  }
 
-    if (vectorDimension == 0 && vectorSearchStrategy != VectorValues.SearchStrategy.NONE) {
-      throw new IllegalStateException(
-          "vector search strategy must be NONE when dimension = 0; got " + vectorSearchStrategy);
+  /**
+   * Verify that the provided docValues type are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  static void verifySameDocValuesType(
+      String fieldName, DocValuesType docValuesType1, DocValuesType docValuesType2) {
+    if (docValuesType1 != docValuesType2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from doc values type="
+              + docValuesType1
+              + " to inconsistent doc values type="
+              + docValuesType2);
     }
+  }
 
-    return true;
+  /**
+   * Verify that the provided store term vectors options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  static void verifySameStoreTermVectors(
+      String fieldName, boolean storeTermVector1, boolean storeTermVector2) {
+    if (storeTermVector1 != storeTermVector2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from storeTermVector="
+              + storeTermVector1
+              + " to inconsistent storeTermVector="
+              + storeTermVector2);
+    }
   }
 
-  // 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);
-      }
+  /**
+   * Verify that the provided omitNorms are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  static void verifySameOmitNorms(String fieldName, boolean omitNorms1, boolean omitNorms2) {
+    if (omitNorms1 != omitNorms2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from omitNorms="
+              + omitNorms1
+              + " to inconsistent omitNorms="
+              + omitNorms2);
     }
+  }
 
-    if (this.pointDimensionCount == 0 && dimensionCount != 0) {
-      this.pointDimensionCount = dimensionCount;
-      this.pointIndexDimensionCount = indexDimensionCount;
-      this.pointNumBytes = dimensionNumBytes;
-    } else if (dimensionCount != 0
-        && (this.pointDimensionCount != dimensionCount
-            || this.pointIndexDimensionCount != indexDimensionCount
-            || this.pointNumBytes != dimensionNumBytes)) {
+  /**
+   * Verify that the provided points indexing options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  static void verifySamePointsOptions(
+      String fieldName,
+      int pointDimensionCount1,
+      int indexDimensionCount1,
+      int numBytes1,
+      int pointDimensionCount2,
+      int indexDimensionCount2,
+      int numBytes2) {
+    if (pointDimensionCount1 != pointDimensionCount2
+        || indexDimensionCount1 != indexDimensionCount2
+        || numBytes1 != numBytes2) {
       throw new IllegalArgumentException(
           "cannot change field \""
-              + name
+              + fieldName
               + "\" from points dimensionCount="
-              + this.pointDimensionCount
+              + pointDimensionCount1
               + ", indexDimensionCount="
-              + this.pointIndexDimensionCount
+              + indexDimensionCount1
               + ", numBytes="
-              + this.pointNumBytes
+              + numBytes1
               + " to inconsistent dimensionCount="
-              + dimensionCount
+              + pointDimensionCount2
               + ", indexDimensionCount="
-              + indexDimensionCount
+              + indexDimensionCount2
               + ", numBytes="
-              + dimensionNumBytes);
+              + numBytes2);
     }
+  }
 
-    // if updated field data is not for indexing, leave the updates out
-    if (this.indexOptions != IndexOptions.NONE) {
-      this.storeTermVector |= storeTermVector; // once vector, always vector
-      this.storePayloads |= storePayloads;
-
-      // Awkward: only drop norms if incoming update is indexed:
-      if (indexOptions != IndexOptions.NONE && this.omitNorms != omitNorms) {
-        this.omitNorms = true; // if one require omitNorms at least once, it remains off for life
-      }
-    }
-    if (this.indexOptions == IndexOptions.NONE
-        || this.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
-      // cannot store payloads if we don't store positions:
-      this.storePayloads = false;
-    }
-    if (attributes != null) {
-      this.attributes.putAll(attributes);
+  /**
+   * Verify that the provided vector indexing options are the same
+   *
+   * @throws IllegalArgumentException if they are not the same
+   */
+  static void verifySameVectorOptions(
+      String fieldName,
+      int vd1,
+      VectorValues.SearchStrategy vst1,
+      int vd2,
+      VectorValues.SearchStrategy vst2) {
+    if (vd1 != vd2 || vst1 != vst2) {
+      throw new IllegalArgumentException(
+          "cannot change field \""
+              + fieldName
+              + "\" from vector dimension="
+              + vd1
+              + ", vector search strategy="
+              + vst1
+              + " to inconsistent vector dimension="
+              + vd2
+              + ", vector search strategy="
+              + vst2);
     }
-    this.checkConsistency();
   }
 
   /**
@@ -353,55 +473,6 @@ public final class FieldInfo {
     return pointNumBytes;
   }
 
-  /**
-   * Record that this field is indexed with vectors, with the specified num of dimensions and
-   * distance function
-   */
-  public void setVectorDimensionAndSearchStrategy(
-      int dimension, VectorValues.SearchStrategy searchStrategy) {
-    if (dimension < 0) {
-      throw new IllegalArgumentException("vector dimension must be >= 0; got " + dimension);
-    }
-    if (dimension > VectorValues.MAX_DIMENSIONS) {
-      throw new IllegalArgumentException(
-          "vector dimension must be <= VectorValues.MAX_DIMENSIONS (="
-              + VectorValues.MAX_DIMENSIONS
-              + "); got "
-              + dimension);
-    }
-    if (dimension == 0 && searchStrategy != VectorValues.SearchStrategy.NONE) {
-      throw new IllegalArgumentException(
-          "vector search strategy must be NONE when the vector dimension = 0; got "
-              + searchStrategy);
-    }
-    if (vectorDimension != 0 && vectorDimension != dimension) {
-      throw new IllegalArgumentException(
-          "cannot change vector dimension from "
-              + vectorDimension
-              + " to "
-              + dimension
-              + " for field=\""
-              + name
-              + "\"");
-    }
-    if (vectorSearchStrategy != VectorValues.SearchStrategy.NONE
-        && vectorSearchStrategy != searchStrategy) {
-      throw new IllegalArgumentException(
-          "cannot change vector search strategy from "
-              + vectorSearchStrategy
-              + " to "
-              + searchStrategy
-              + " for field=\""
-              + name
-              + "\"");
-    }
-
-    this.vectorDimension = dimension;
-    this.vectorSearchStrategy = searchStrategy;
-
-    assert checkConsistency();
-  }
-
   /** Returns the number of dimensions of the vector value */
   public int getVectorDimension() {
     return vectorDimension;
@@ -438,28 +509,22 @@ public final class FieldInfo {
     return indexOptions;
   }
 
-  /** Record the {@link IndexOptions} to use with this field. */
-  public void setIndexOptions(IndexOptions newIndexOptions) {
-    if (indexOptions != newIndexOptions) {
-      if (indexOptions == IndexOptions.NONE) {
-        indexOptions = newIndexOptions;
-      } else if (newIndexOptions != IndexOptions.NONE) {
-        throw new IllegalArgumentException(
-            "cannot change field \""
-                + name
-                + "\" from index options="
-                + indexOptions
-                + " to inconsistent index options="
-                + newIndexOptions);
-      }
-    }
+  /**
+   * Returns name of this field
+   *
+   * @return name
+   */
+  public String getName() {
+    return name;
+  }
 
-    if (indexOptions == IndexOptions.NONE
-        || indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
-      // cannot store payloads if we don't store positions:
-      storePayloads = false;
-    }
-    this.checkConsistency();
+  /**
+   * Returns the field number
+   *
+   * @return field number
+   */
+  public int getFieldNumber() {
+    return number;
   }
 
   /**
diff --git a/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java b/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
index 0f8d341..c9d3028 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
@@ -16,6 +16,13 @@
  */
 package org.apache.lucene.index;
 
+import static org.apache.lucene.index.FieldInfo.verifySameDocValuesType;
+import static org.apache.lucene.index.FieldInfo.verifySameIndexOptions;
+import static org.apache.lucene.index.FieldInfo.verifySameOmitNorms;
+import static org.apache.lucene.index.FieldInfo.verifySamePointsOptions;
+import static org.apache.lucene.index.FieldInfo.verifySameStoreTermVectors;
+import static org.apache.lucene.index.FieldInfo.verifySameVectorOptions;
+
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -169,7 +176,9 @@ public class FieldInfos implements Iterable<FieldInfo> {
               .orElse(null);
       final Builder builder = new Builder(new FieldNumbers(softDeletesField));
       for (final LeafReaderContext ctx : leaves) {
-        builder.add(ctx.reader().getFieldInfos());
+        for (FieldInfo fieldInfo : ctx.reader().getFieldInfos()) {
+          builder.add(fieldInfo);
+        }
       }
       return builder.finish();
     }
@@ -311,6 +320,8 @@ public class FieldInfos implements Iterable<FieldInfo> {
     private final Map<String, FieldDimensions> dimensions;
 
     private final Map<String, FieldVectorProperties> vectorProps;
+    private final Map<String, Boolean> omitNorms;
+    private final Map<String, Boolean> storeTermVectors;
 
     // TODO: we should similarly catch an attempt to turn
     // norms back on after they were already committed; today
@@ -327,6 +338,8 @@ public class FieldInfos implements Iterable<FieldInfo> {
       this.docValuesType = new HashMap<>();
       this.dimensions = new HashMap<>();
       this.vectorProps = new HashMap<>();
+      this.omitNorms = new HashMap<>();
+      this.storeTermVectors = new HashMap<>();
       this.softDeletesFieldName = softDeletesFieldName;
     }
 
@@ -335,116 +348,16 @@ public class FieldInfos implements Iterable<FieldInfo> {
      * tries to add it with the given preferred field number assigned if possible otherwise the
      * first unassigned field number is used as the field number.
      */
-    synchronized int addOrGet(
-        String fieldName,
-        int preferredFieldNumber,
-        IndexOptions indexOptions,
-        DocValuesType dvType,
-        int dimensionCount,
-        int indexDimensionCount,
-        int dimensionNumBytes,
-        int vectorDimension,
-        VectorValues.SearchStrategy searchStrategy,
-        boolean isSoftDeletesField) {
-      if (indexOptions != IndexOptions.NONE) {
-        IndexOptions currentOpts = this.indexOptions.get(fieldName);
-        if (currentOpts == null) {
-          this.indexOptions.put(fieldName, indexOptions);
-        } else if (currentOpts != IndexOptions.NONE && currentOpts != indexOptions) {
-          throw new IllegalArgumentException(
-              "cannot change field \""
-                  + fieldName
-                  + "\" from index options="
-                  + currentOpts
-                  + " to inconsistent index options="
-                  + indexOptions);
-        }
-      }
-      if (dvType != DocValuesType.NONE) {
-        DocValuesType currentDVType = docValuesType.get(fieldName);
-        if (currentDVType == null) {
-          docValuesType.put(fieldName, dvType);
-        } else if (currentDVType != DocValuesType.NONE && currentDVType != dvType) {
-          throw new IllegalArgumentException(
-              "cannot change DocValues type from "
-                  + currentDVType
-                  + " to "
-                  + dvType
-                  + " for field \""
-                  + fieldName
-                  + "\"");
-        }
-      }
-      if (dimensionCount != 0) {
-        FieldDimensions dims = dimensions.get(fieldName);
-        if (dims != null) {
-          if (dims.dimensionCount != dimensionCount) {
-            throw new IllegalArgumentException(
-                "cannot change point dimension count from "
-                    + dims.dimensionCount
-                    + " to "
-                    + dimensionCount
-                    + " for field=\""
-                    + fieldName
-                    + "\"");
-          }
-          if (dims.indexDimensionCount != indexDimensionCount) {
-            throw new IllegalArgumentException(
-                "cannot change point index dimension count from "
-                    + dims.indexDimensionCount
-                    + " to "
-                    + indexDimensionCount
-                    + " for field=\""
-                    + fieldName
-                    + "\"");
-          }
-          if (dims.dimensionNumBytes != dimensionNumBytes) {
-            throw new IllegalArgumentException(
-                "cannot change point numBytes from "
-                    + dims.dimensionNumBytes
-                    + " to "
-                    + dimensionNumBytes
-                    + " for field=\""
-                    + fieldName
-                    + "\"");
-          }
-        } else {
-          dimensions.put(
-              fieldName,
-              new FieldDimensions(dimensionCount, indexDimensionCount, dimensionNumBytes));
-        }
-      }
-      if (vectorDimension != 0) {
-        FieldVectorProperties props = vectorProps.get(fieldName);
-        if (props != null) {
-          if (props.numDimensions != vectorDimension) {
-            throw new IllegalArgumentException(
-                "cannot change vector dimension from "
-                    + props.numDimensions
-                    + " to "
-                    + vectorDimension
-                    + " for field=\""
-                    + fieldName
-                    + "\"");
-          }
-          if (props.searchStrategy != searchStrategy) {
-            throw new IllegalArgumentException(
-                "cannot change vector search strategy from "
-                    + props.searchStrategy
-                    + " to "
-                    + searchStrategy
-                    + " for field=\""
-                    + fieldName
-                    + "\"");
-          }
-        } else {
-          vectorProps.put(fieldName, new FieldVectorProperties(vectorDimension, searchStrategy));
-        }
-      }
+    synchronized int addOrGet(FieldInfo fi) {
+      String fieldName = fi.getName();
+      verifySoftDeletedFieldName(fieldName, fi.isSoftDeletesField());
       Integer fieldNumber = nameToNumber.get(fieldName);
-      if (fieldNumber == null) {
-        final Integer preferredBoxed = Integer.valueOf(preferredFieldNumber);
-        if (preferredFieldNumber != -1 && !numberToName.containsKey(preferredBoxed)) {
+
+      if (fieldNumber != null) {
+        verifySameSchema(fi);
+      } else { // first time we see this field in this index
+        final Integer preferredBoxed = Integer.valueOf(fi.number);
+        if (fi.number != -1 && !numberToName.containsKey(preferredBoxed)) {
           // cool - we can use this number globally
           fieldNumber = preferredBoxed;
         } else {
@@ -457,8 +370,26 @@ public class FieldInfos implements Iterable<FieldInfo> {
         assert fieldNumber >= 0;
         numberToName.put(fieldNumber, fieldName);
         nameToNumber.put(fieldName, fieldNumber);
+        this.indexOptions.put(fieldName, fi.getIndexOptions());
+        if (fi.getIndexOptions() != IndexOptions.NONE) {
+          this.storeTermVectors.put(fieldName, fi.hasVectors());
+          this.omitNorms.put(fieldName, fi.omitsNorms());
+        }
+        docValuesType.put(fieldName, fi.getDocValuesType());
+        dimensions.put(
+            fieldName,
+            new FieldDimensions(
+                fi.getPointDimensionCount(),
+                fi.getPointIndexDimensionCount(),
+                fi.getPointNumBytes()));
+        vectorProps.put(
+            fieldName,
+            new FieldVectorProperties(fi.getVectorDimension(), fi.getVectorSearchStrategy()));
       }
+      return fieldNumber.intValue();
+    }
 
+    private void verifySoftDeletedFieldName(String fieldName, boolean isSoftDeletesField) {
       if (isSoftDeletesField) {
         if (softDeletesFieldName == null) {
           throw new IllegalArgumentException(
@@ -481,205 +412,163 @@ public class FieldInfos implements Iterable<FieldInfo> {
                 + 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);
-      }
-    }
+    private void verifySameSchema(FieldInfo fi) {
+      String fieldName = fi.getName();
+      IndexOptions currentOpts = this.indexOptions.get(fieldName);
+      verifySameIndexOptions(fieldName, currentOpts, fi.getIndexOptions());
+      if (currentOpts != IndexOptions.NONE) {
+        boolean curStoreTermVector = this.storeTermVectors.get(fieldName);
+        verifySameStoreTermVectors(fieldName, curStoreTermVector, fi.hasVectors());
+        boolean curOmitNorms = this.omitNorms.get(fieldName);
+        verifySameOmitNorms(fieldName, curOmitNorms, fi.omitsNorms());
+      }
+
+      DocValuesType currentDVType = docValuesType.get(fieldName);
+      verifySameDocValuesType(fieldName, currentDVType, fi.getDocValuesType());
+
+      FieldDimensions dims = dimensions.get(fieldName);
+      verifySamePointsOptions(
+          fieldName,
+          dims.dimensionCount,
+          dims.indexDimensionCount,
+          dims.dimensionNumBytes,
+          fi.getPointDimensionCount(),
+          fi.getPointIndexDimensionCount(),
+          fi.getPointNumBytes());
 
-    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
-                + "\"");
-      }
+      FieldVectorProperties props = vectorProps.get(fieldName);
+      verifySameVectorOptions(
+          fieldName,
+          props.numDimensions,
+          props.searchStrategy,
+          fi.getVectorDimension(),
+          fi.getVectorSearchStrategy());
     }
 
-    synchronized void verifyConsistentDimensions(
-        Integer number,
-        String name,
-        int dataDimensionCount,
-        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) {
+    /**
+     * This function is called from {@code IndexWriter} to verify if doc values of the field can be
+     * updated. If the field with this name already exists, we verify that it is doc values only
+     * field. If the field doesn't exists and the parameter fieldMustExist is false, we create a new
+     * field in the global field numbers.
+     *
+     * @param fieldName - name of the field
+     * @param dvType - expected doc values type
+     * @param fieldMustExist – if the field must exist.
+     * @throws IllegalArgumentException if the field must exist, but it doesn't, or if the field
+     *     exists, but it is not doc values only field with the provided doc values type.
+     */
+    synchronized void verifyOrCreateDvOnlyField(
+        String fieldName, DocValuesType dvType, boolean fieldMustExist) {
+      if (nameToNumber.containsKey(fieldName) == false) {
+        if (fieldMustExist) {
           throw new IllegalArgumentException(
-              "cannot change point dimension count from "
-                  + dim.dimensionCount
-                  + " to "
-                  + dataDimensionCount
-                  + " for field=\""
-                  + name
-                  + "\"");
+              "Can't update ["
+                  + dvType
+                  + "] doc values; the field ["
+                  + fieldName
+                  + "] doesn't exist.");
+        } else {
+          // create dv only field
+          FieldInfo fi =
+              new FieldInfo(
+                  fieldName,
+                  -1,
+                  false,
+                  false,
+                  false,
+                  IndexOptions.NONE,
+                  dvType,
+                  -1,
+                  new HashMap<>(),
+                  0,
+                  0,
+                  0,
+                  0,
+                  VectorValues.SearchStrategy.NONE,
+                  (softDeletesFieldName != null && softDeletesFieldName.equals(fieldName)));
+          addOrGet(fi);
         }
-        if (dim.indexDimensionCount != indexDimensionCount) {
+      } else {
+        // verify that field is doc values only field with the give doc values type
+        DocValuesType fieldDvType = docValuesType.get(fieldName);
+        if (dvType != fieldDvType) {
           throw new IllegalArgumentException(
-              "cannot change point index dimension count from "
-                  + dim.indexDimensionCount
-                  + " to "
-                  + indexDimensionCount
-                  + " for field=\""
-                  + name
-                  + "\"");
+              "Can't update ["
+                  + dvType
+                  + "] doc values; the field ["
+                  + fieldName
+                  + "] has inconsistent doc values' type of ["
+                  + fieldDvType
+                  + "].");
         }
-        if (dim.dimensionNumBytes != dimensionNumBytes) {
+        FieldDimensions fdimensions = dimensions.get(fieldName);
+        if (fdimensions != null && fdimensions.dimensionCount != 0) {
           throw new IllegalArgumentException(
-              "cannot change point numBytes from "
-                  + dim.dimensionNumBytes
-                  + " to "
-                  + dimensionNumBytes
-                  + " for field=\""
-                  + name
-                  + "\"");
+              "Can't update ["
+                  + dvType
+                  + "] doc values; the field ["
+                  + fieldName
+                  + "] must be doc values only field, but is also indexed with points.");
         }
-      }
-    }
-
-    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) {
+        IndexOptions ioptions = indexOptions.get(fieldName);
+        if (ioptions != null && ioptions != IndexOptions.NONE) {
           throw new IllegalArgumentException(
-              "cannot change vector dimension from "
-                  + props.numDimensions
-                  + " to "
-                  + numDimensions
-                  + " for field=\""
-                  + name
-                  + "\"");
+              "Can't update ["
+                  + dvType
+                  + "] doc values; the field ["
+                  + fieldName
+                  + "] must be doc values only field, but is also indexed with postings.");
         }
-        if (props.searchStrategy != searchStrategy) {
+        FieldVectorProperties fvp = vectorProps.get(fieldName);
+        if (fvp != null && fvp.numDimensions != 0) {
           throw new IllegalArgumentException(
-              "cannot change vector search strategy from "
-                  + props.searchStrategy
-                  + " to "
-                  + searchStrategy
-                  + " for field=\""
-                  + name
-                  + "\"");
+              "Can't update ["
+                  + dvType
+                  + "] doc values; the field ["
+                  + fieldName
+                  + "] must be doc values only field, but is also indexed with vectors.");
         }
       }
     }
 
     /**
-     * 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);
+      return new FieldInfo(
+          fieldName,
+          newFieldNumber,
+          false,
+          false,
+          false,
+          IndexOptions.NONE,
+          dvType,
+          -1,
+          new HashMap<>(),
+          0,
+          0,
+          0,
+          0,
+          VectorValues.SearchStrategy.NONE,
+          isSoftDeletesField);
     }
 
     synchronized Set<String> getFieldNames() {
@@ -694,85 +583,6 @@ public class FieldInfos implements Iterable<FieldInfo> {
       dimensions.clear();
       lowestUnassignedFieldNumber = -1;
     }
-
-    synchronized void setIndexOptions(int number, String name, IndexOptions indexOptions) {
-      verifyConsistent(number, name, indexOptions);
-      this.indexOptions.put(name, indexOptions);
-    }
-
-    synchronized void setDocValuesType(int number, String name, DocValuesType dvType) {
-      verifyConsistent(number, name, dvType);
-      docValuesType.put(name, dvType);
-    }
-
-    synchronized void setDimensions(
-        int number,
-        String name,
-        int dimensionCount,
-        int indexDimensionCount,
-        int dimensionNumBytes) {
-      if (dimensionCount > PointValues.MAX_DIMENSIONS) {
-        throw new IllegalArgumentException(
-            "dimensionCount must be <= PointValues.MAX_DIMENSIONS (= "
-                + PointValues.MAX_DIMENSIONS
-                + "); got "
-                + dimensionCount
-                + " for field=\""
-                + name
-                + "\"");
-      }
-      if (dimensionNumBytes > PointValues.MAX_NUM_BYTES) {
-        throw new IllegalArgumentException(
-            "dimension numBytes must be <= PointValues.MAX_NUM_BYTES (= "
-                + PointValues.MAX_NUM_BYTES
-                + "); got "
-                + dimensionNumBytes
-                + " for field=\""
-                + name
-                + "\"");
-      }
-      if (indexDimensionCount > dimensionCount) {
-        throw new IllegalArgumentException(
-            "indexDimensionCount must be <= dimensionCount (= "
-                + dimensionCount
-                + "); got "
-                + indexDimensionCount
-                + " for field=\""
-                + name
-                + "\"");
-      }
-      if (indexDimensionCount > PointValues.MAX_INDEX_DIMENSIONS) {
-        throw new IllegalArgumentException(
-            "indexDimensionCount must be <= PointValues.MAX_INDEX_DIMENSIONS (= "
-                + PointValues.MAX_INDEX_DIMENSIONS
-                + "); got "
-                + indexDimensionCount
-                + " for field=\""
-                + name
-                + "\"");
-      }
-      verifyConsistentDimensions(
-          number, name, dimensionCount, indexDimensionCount, dimensionNumBytes);
-      dimensions.put(
-          name, new FieldDimensions(dimensionCount, indexDimensionCount, dimensionNumBytes));
-    }
-
-    synchronized void setVectorDimensionsAndSearchStrategy(
-        int number, String name, int numDimensions, VectorValues.SearchStrategy searchStrategy) {
-      if (numDimensions <= 0) {
-        throw new IllegalArgumentException(
-            "vector numDimensions must be > 0; got " + numDimensions);
-      }
-      if (numDimensions > VectorValues.MAX_DIMENSIONS) {
-        throw new IllegalArgumentException(
-            "vector numDimensions must be <= VectorValues.MAX_DIMENSIONS (="
-                + VectorValues.MAX_DIMENSIONS
-                + "); got "
-                + numDimensions);
-      }
-      verifyConsistentVectorProperties(number, name, numDimensions, searchStrategy);
-      vectorProps.put(name, new FieldVectorProperties(numDimensions, searchStrategy));
-    }
   }
 
   static final class Builder {
@@ -786,177 +596,92 @@ public class FieldInfos implements Iterable<FieldInfo> {
       this.globalFieldNumbers = globalFieldNumbers;
     }
 
-    public void add(FieldInfos other) {
-      assert assertNotFinished();
-      for (FieldInfo fieldInfo : other) {
-        add(fieldInfo);
-      }
-    }
-
-    /** 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);
-      }
-
-      return fi;
-    }
-
-    private FieldInfo addOrUpdateInternal(
-        String name,
-        int preferredFieldNumber,
-        boolean storeTermVector,
-        boolean omitNorms,
-        boolean storePayloads,
-        IndexOptions indexOptions,
-        DocValuesType docValues,
-        long dvGen,
-        Map<String, String> attributes,
-        int dataDimensionCount,
-        int indexDimensionCount,
-        int dimensionNumBytes,
-        int vectorDimension,
-        VectorValues.SearchStrategy vectorSearchStrategy,
-        boolean isSoftDeletesField) {
-      assert assertNotFinished();
-      if (docValues == null) {
-        throw new NullPointerException("DocValuesType must not be null");
-      }
-      if (attributes != null) {
-        // original attributes is UnmodifiableMap
-        attributes = new HashMap<>(attributes);
-      }
-
-      FieldInfo fi = fieldInfo(name);
-      if (fi == null) {
-        // 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 int fieldNumber =
-            globalFieldNumbers.addOrGet(
-                name,
-                preferredFieldNumber,
-                indexOptions,
-                docValues,
-                dataDimensionCount,
-                indexDimensionCount,
-                dimensionNumBytes,
-                vectorDimension,
-                vectorSearchStrategy,
-                isSoftDeletesField);
-        fi =
-            new FieldInfo(
-                name,
-                fieldNumber,
-                storeTermVector,
-                omitNorms,
-                storePayloads,
-                indexOptions,
-                docValues,
-                dvGen,
-                attributes,
-                dataDimensionCount,
-                indexDimensionCount,
-                dimensionNumBytes,
-                vectorDimension,
-                vectorSearchStrategy,
-                isSoftDeletesField);
-        assert !byName.containsKey(fi.name);
-        globalFieldNumbers.verifyConsistent(
-            Integer.valueOf(fi.number), fi.name, fi.getDocValuesType());
-        byName.put(fi.name, fi);
-      } else {
-        fi.update(
-            storeTermVector,
-            omitNorms,
-            storePayloads,
-            indexOptions,
-            attributes,
-            dataDimensionCount,
-            indexDimensionCount,
-            dimensionNumBytes);
-
-        if (docValues != DocValuesType.NONE) {
-          // Only pay the synchronization cost if fi does not already have a DVType
-          boolean updateGlobal = fi.getDocValuesType() == DocValuesType.NONE;
-          if (updateGlobal) {
-            // Must also update docValuesType map so it's
-            // aware of this field's DocValuesType.  This will throw IllegalArgumentException if
-            // an illegal type change was attempted.
-            globalFieldNumbers.setDocValuesType(fi.number, name, docValues);
-          }
-
-          fi.setDocValuesType(docValues); // this will also perform the consistency check.
-          fi.setDocValuesGen(dvGen);
-        }
-      }
-      return fi;
+    public String getSoftDeletesFieldName() {
+      return globalFieldNumbers.softDeletesFieldName;
     }
 
+    /**
+     * Adds the provided FieldInfo to this Builder if this field doesn't exist in this Builder. Also
+     * adds a new field with its schema options to the global FieldNumbers if the field doesn't
+     * exist globally in the index. The field number is reused if possible for consistent field
+     * numbers across segments.
+     *
+     * <p>If the field already exists: 1) the provided FieldInfo's schema is checked against the
+     * existing field and 2) the provided FieldInfo's attributes are added to the existing
+     * FieldInfo's attributes.
+     *
+     * @param fi – FieldInfo to add
+     * @return The existing FieldInfo if the field with this name already exists in the builder, or
+     *     a new constructed FieldInfo with the same schema as provided and a consistent global
+     *     field number.
+     * @throws IllegalArgumentException if there already exists field with this name in Builder but
+     *     with a different schema
+     * @throws IllegalArgumentException if there already exists field with this name globally but
+     *     with a different schema.
+     * @throws IllegalStateException if the Builder is already finished building and doesn't accept
+     *     new fields.
+     */
     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
-      return addOrUpdateInternal(
-          fi.name,
-          fi.number,
-          fi.hasVectors(),
-          fi.omitsNorms(),
-          fi.hasPayloads(),
-          fi.getIndexOptions(),
-          fi.getDocValuesType(),
-          dvGen,
-          fi.attributes(),
-          fi.getPointDimensionCount(),
-          fi.getPointIndexDimensionCount(),
-          fi.getPointNumBytes(),
-          fi.getVectorDimension(),
-          fi.getVectorSearchStrategy(),
-          fi.isSoftDeletesField());
+    /**
+     * Adds the provided FieldInfo with the provided dvGen to this Builder if this field doesn't
+     * exist in this Builder. Also adds a new field with its schema options to the global
+     * FieldNumbers if the field doesn't exist globally in the index. The field number is reused if
+     * possible for consistent field numbers across segments.
+     *
+     * <p>If the field already exists: 1) the provided FieldInfo's schema is checked against the
+     * existing field and 2) the provided FieldInfo's attributes are added to the existing
+     * FieldInfo's attributes.
+     *
+     * @param fi – FieldInfo to add
+     * @param dvGen – doc values generation of the FieldInfo to add
+     * @return The existing FieldInfo if the field with this name already exists in the builder, or
+     *     a new constructed FieldInfo with the same schema as provided and a consistent global
+     *     field number.
+     * @throws IllegalArgumentException if there already exists field with this name in Builder but
+     *     with a different schema
+     * @throws IllegalArgumentException if there already exists field with this name globally but
+     *     with a different schema.
+     * @throws IllegalStateException if the Builder is already finished building and doesn't accept
+     *     new fields.
+     */
+    FieldInfo add(FieldInfo fi, long dvGen) {
+      final FieldInfo curFi = fieldInfo(fi.getName());
+      if (curFi != null) {
+        curFi.verifySameSchema(fi);
+        if (fi.attributes() != null) {
+          fi.attributes().forEach((k, v) -> curFi.putAttribute(k, v));
+        }
+        return curFi;
+      }
+      // This field wasn't yet added to this in-RAM segment's FieldInfo,
+      // so now we get a global number for this field.
+      // If the field was seen before then we'll get the same name and number,
+      // else we'll allocate a new one
+      assert assertNotFinished();
+      final int fieldNumber = globalFieldNumbers.addOrGet(fi);
+      FieldInfo fiNew =
+          new FieldInfo(
+              fi.getName(),
+              fieldNumber,
+              fi.hasVectors(),
+              fi.omitsNorms(),
+              fi.hasPayloads(),
+              fi.getIndexOptions(),
+              fi.getDocValuesType(),
+              dvGen,
+              // original attributes is UnmodifiableMap
+              new HashMap<>(fi.attributes()),
+              fi.getPointDimensionCount(),
+              fi.getPointIndexDimensionCount(),
+              fi.getPointNumBytes(),
+              fi.getVectorDimension(),
+              fi.getVectorSearchStrategy(),
+              fi.isSoftDeletesField());
+      byName.put(fiNew.getName(), fiNew);
+      return fiNew;
     }
 
     public FieldInfo fieldInfo(String fieldName) {
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
index 8f266b7..9aa42d5 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -1259,17 +1259,7 @@ public class IndexWriter
     for (SegmentCommitInfo info : segmentInfos) {
       FieldInfos fis = readFieldInfos(info);
       for (FieldInfo fi : fis) {
-        map.addOrGet(
-            fi.name,
-            fi.number,
-            fi.getIndexOptions(),
-            fi.getDocValuesType(),
-            fi.getPointDimensionCount(),
-            fi.getPointIndexDimensionCount(),
-            fi.getPointNumBytes(),
-            fi.getVectorDimension(),
-            fi.getVectorSearchStrategy(),
-            fi.isSoftDeletesField());
+        map.addOrGet(fi);
       }
     }
 
@@ -1861,7 +1851,7 @@ public class IndexWriter
   /**
    * Updates a document's {@link NumericDocValues} for <code>field</code> to the given <code>value
    * </code>. You can only update fields that already exist in the index, not add new fields through
-   * this method.
+   * this method. You can only update fields that were indexed with doc values only.
    *
    * @param term the term to identify the document(s) to be updated
    * @param field field name of the {@link NumericDocValues} field
@@ -1872,9 +1862,7 @@ public class IndexWriter
    */
   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!");
-    }
+    globalFieldNumberMap.verifyOrCreateDvOnlyField(field, DocValuesType.NUMERIC, true);
     if (config.getIndexSortFields().contains(field)) {
       throw new IllegalArgumentException(
           "cannot update docvalues field involved in the index sort, field="
@@ -1894,7 +1882,7 @@ public class IndexWriter
   /**
    * Updates a document's {@link BinaryDocValues} for <code>field</code> to the given <code>value
    * </code>. You can only update fields that already exist in the index, not add new fields through
-   * this method.
+   * this method. You can only update fields that were indexed only with doc values.
    *
    * <p><b>NOTE:</b> this method currently replaces the existing value of all affected documents
    * with the new value.
@@ -1911,9 +1899,7 @@ public class IndexWriter
     if (value == null) {
       throw new IllegalArgumentException("cannot update a field to a null value: " + field);
     }
-    if (!globalFieldNumberMap.contains(field, DocValuesType.BINARY)) {
-      throw new IllegalArgumentException("can only update existing binary-docvalues fields!");
-    }
+    globalFieldNumberMap.verifyOrCreateDvOnlyField(field, DocValuesType.BINARY, true);
     try {
       return maybeProcessEvents(
           docWriter.updateDocValues(new BinaryDocValuesUpdate(term, field, value)));
@@ -1958,23 +1944,10 @@ public class IndexWriter
         throw new IllegalArgumentException(
             "can only update NUMERIC or BINARY fields! field=" + f.name());
       }
-      if (globalFieldNumberMap.contains(f.name(), dvType) == false) {
-        // if this field doesn't exists we try to add it. if it exists and the DV type doesn't match
-        // we
-        // get a consistent error message as if you try to do that during an indexing operation.
-        globalFieldNumberMap.addOrGet(
-            f.name(),
-            -1,
-            IndexOptions.NONE,
-            dvType,
-            0,
-            0,
-            0,
-            0,
-            VectorValues.SearchStrategy.NONE,
-            f.name().equals(config.softDeletesField));
-        assert globalFieldNumberMap.contains(f.name(), dvType);
-      }
+      // if this field doesn't exists we try to add it.
+      // if it exists and the DV type doesn't match or it is not DV only field,
+      // we will get an error.
+      globalFieldNumberMap.verifyOrCreateDvOnlyField(f.name(), dvType, false);
       if (config.getIndexSortFields().contains(f.name())) {
         throw new IllegalArgumentException(
             "cannot update docvalues field involved in the index sort, field="
@@ -3040,19 +3013,9 @@ public class IndexWriter
 
             FieldInfos fis = readFieldInfos(info);
             for (FieldInfo fi : fis) {
-              // This will throw exceptions if any of the incoming fields have an illegal schema
-              // change:
-              globalFieldNumberMap.addOrGet(
-                  fi.name,
-                  fi.number,
-                  fi.getIndexOptions(),
-                  fi.getDocValuesType(),
-                  fi.getPointDimensionCount(),
-                  fi.getPointIndexDimensionCount(),
-                  fi.getPointNumBytes(),
-                  fi.getVectorDimension(),
-                  fi.getVectorSearchStrategy(),
-                  fi.isSoftDeletesField());
+              // This will throw exceptions if any of the incoming fields
+              // has an illegal schema change
+              globalFieldNumberMap.addOrGet(fi);
             }
             infos.add(copySegmentAsIs(info, newSegName, context));
           }
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexingChain.java b/lucene/core/src/java/org/apache/lucene/index/IndexingChain.java
index 9f3a67e..326be95 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexingChain.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexingChain.java
@@ -79,6 +79,7 @@ final class IndexingChain implements Accountable {
 
   // Holds fields seen in each document
   private PerField[] fields = new PerField[1];
+  private PerField[] docFields = new PerField[2];
   private final InfoStream infoStream;
   private final ByteBlockPool.Allocator byteBlockAllocator;
   private final LiveIndexWriterConfig indexWriterConfig;
@@ -326,37 +327,22 @@ final class IndexingChain implements Accountable {
         PerField perField = fieldHash[i];
         while (perField != null) {
           if (perField.pointValuesWriter != null) {
-            if (perField.fieldInfo.getPointDimensionCount() == 0) {
-              // BUG
-              throw new AssertionError(
-                  "segment="
-                      + state.segmentInfo
-                      + ": field=\""
-                      + perField.fieldInfo.name
-                      + "\" has no points but wrote them");
-            }
-            if (pointsWriter == null) {
-              // lazy init
-              PointsFormat fmt = state.segmentInfo.getCodec().pointsFormat();
-              if (fmt == null) {
-                throw new IllegalStateException(
-                    "field=\""
-                        + perField.fieldInfo.name
-                        + "\" was indexed as points but codec does not support points");
+            // We could have initialized pointValuesWriter, but failed to write even a single doc
+            if (perField.pointValuesWriter.getNumDocs() > 0) {
+              if (pointsWriter == null) {
+                // lazy init
+                PointsFormat fmt = state.segmentInfo.getCodec().pointsFormat();
+                if (fmt == null) {
+                  throw new IllegalStateException(
+                      "field=\""
+                          + perField.fieldInfo.name
+                          + "\" was indexed as points but codec does not support points");
+                }
+                pointsWriter = fmt.fieldsWriter(state);
               }
-              pointsWriter = fmt.fieldsWriter(state);
+              perField.pointValuesWriter.flush(state, sortMap, pointsWriter);
             }
-
-            perField.pointValuesWriter.flush(state, sortMap, pointsWriter);
             perField.pointValuesWriter = null;
-          } else if (perField.fieldInfo.getPointDimensionCount() != 0) {
-            // BUG
-            throw new AssertionError(
-                "segment="
-                    + state.segmentInfo
-                    + ": field=\""
-                    + perField.fieldInfo.name
-                    + "\" has points but did not write them");
           }
           perField = perField.next;
         }
@@ -399,7 +385,8 @@ final class IndexingChain implements Accountable {
             }
             perField.docValuesWriter.flush(state, sortMap, dvConsumer);
             perField.docValuesWriter = null;
-          } else if (perField.fieldInfo.getDocValuesType() != DocValuesType.NONE) {
+          } else if (perField.fieldInfo != null
+              && perField.fieldInfo.getDocValuesType() != DocValuesType.NONE) {
             // BUG
             throw new AssertionError(
                 "segment="
@@ -470,7 +457,7 @@ final class IndexingChain implements Accountable {
 
             perField.vectorValuesWriter.flush(sortMap, vectorWriter);
             perField.vectorValuesWriter = null;
-          } else if (perField.fieldInfo.getVectorDimension() != 0) {
+          } else if (perField.fieldInfo != null && perField.fieldInfo.getVectorDimension() != 0) {
             // BUG
             throw new AssertionError(
                 "segment="
@@ -548,7 +535,7 @@ final class IndexingChain implements Accountable {
     for (int j = 0; j < fieldHash.length; j++) {
       PerField fp0 = fieldHash[j];
       while (fp0 != null) {
-        final int hashPos2 = fp0.fieldInfo.name.hashCode() & newHashMask;
+        final int hashPos2 = fp0.fieldName.hashCode() & newHashMask;
         PerField nextFP0 = fp0.next;
         fp0.next = newHashArray[hashPos2];
         newHashArray[hashPos2] = fp0;
@@ -581,12 +568,11 @@ final class IndexingChain implements Accountable {
   }
 
   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):
+    // number of unique fields by names (collapses multiple field instances by the same name)
     int fieldCount = 0;
-
+    int indexedFieldCount = 0; // number of unique fields indexed with postings
     long fieldGen = nextFieldGen++;
+    int docFieldIdx = 0;
 
     // NOTE: we need two passes here, in case there are
     // multi-valued fields, because we must process all
@@ -594,65 +580,159 @@ final class IndexingChain implements Accountable {
     // analyzer is free to reuse TokenStream across fields
     // (i.e., we cannot have more than one TokenStream
     // running "at once"):
-
     termsHash.startDocument();
-
     startStoredFields(docID);
     try {
+      // 1st pass over doc fields – verify that doc schema matches the index schema
+      // build schema for each unique doc field
+      for (IndexableField field : document) {
+        IndexableFieldType fieldType = field.fieldType();
+        PerField pf = getOrAddPerField(field.name(), fieldType);
+        if (pf.fieldGen != fieldGen) { // first time we see this field in this document
+          fields[fieldCount++] = pf;
+          pf.fieldGen = fieldGen;
+          pf.reset(docID);
+        }
+        if (docFieldIdx >= docFields.length) oversizeDocFields();
+        docFields[docFieldIdx++] = pf;
+        updateDocFieldSchema(field.name(), pf.schema, fieldType);
+      }
+      // For each field, if it the first time we see this field in this segment,
+      // initialize its FieldInfo.
+      // If we have already seen this field, verify that its schema
+      // within the current doc matches its schema in the index.
+      for (int i = 0; i < fieldCount; i++) {
+        PerField pf = fields[i];
+        if (pf.fieldInfo == null) {
+          initializeFieldInfo(pf);
+        } else {
+          pf.schema.assertSameSchema(pf.fieldInfo);
+        }
+      }
+
+      // 2nd pass over doc fields – index each field
+      // also count the number of unique fields indexed with postings
+      docFieldIdx = 0;
       for (IndexableField field : document) {
-        fieldCount = processField(docID, field, fieldGen, fieldCount);
+        if (processField(docID, field, docFields[docFieldIdx])) {
+          fields[indexedFieldCount] = docFields[docFieldIdx];
+          indexedFieldCount++;
+        }
+        docFieldIdx++;
       }
     } finally {
       if (hasHitAbortingException == false) {
         // Finish each indexed field name seen in the document:
-        for (int i = 0; i < fieldCount; i++) {
+        for (int i = 0; i < indexedFieldCount; i++) {
           fields[i].finish(docID);
         }
         finishStoredFields();
+        // TODO: for broken docs, optimize termsHash.finishDocument
+        try {
+          termsHash.finishDocument(docID);
+        } catch (Throwable th) {
+          // Must abort, on the possibility that on-disk term
+          // vectors are now corrupt:
+          abortingExceptionConsumer.accept(th);
+          throw th;
+        }
       }
     }
+  }
 
-    try {
-      termsHash.finishDocument(docID);
-    } catch (Throwable th) {
-      // Must abort, on the possibility that on-disk term
-      // vectors are now corrupt:
-      abortingExceptionConsumer.accept(th);
-      throw th;
+  private void oversizeDocFields() {
+    PerField[] newDocFields =
+        new PerField
+            [ArrayUtil.oversize(docFields.length + 1, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+    System.arraycopy(docFields, 0, newDocFields, 0, docFields.length);
+    docFields = newDocFields;
+  }
+
+  private void initializeFieldInfo(PerField pf) throws IOException {
+    // Create and add a new fieldInfo to fieldInfos for this segment.
+    // During the creation of FieldInfo there is also verification of the correctness of all its
+    // parameters.
+
+    // If the fieldInfo doesn't exist in globalFieldNumbers for the whole index,
+    // it will be added there.
+    // If the field already exists in globalFieldNumbers (i.e. field present in other segments),
+    // we check consistency of its schema with schema for the whole index.
+    FieldSchema s = pf.schema;
+    if (indexWriterConfig.getIndexSort() != null && s.docValuesType != DocValuesType.NONE) {
+      final Sort indexSort = indexWriterConfig.getIndexSort();
+      validateIndexSortDVType(indexSort, pf.fieldName, s.docValuesType);
+    }
+    FieldInfo fi =
+        fieldInfos.add(
+            new FieldInfo(
+                pf.fieldName,
+                -1,
+                s.storeTermVector,
+                s.omitNorms,
+                // storePayloads is set up during indexing, if payloads were seen
+                false,
+                s.indexOptions,
+                s.docValuesType,
+                s.dvGen,
+                s.attributes,
+                s.pointDimensionCount,
+                s.pointIndexDimensionCount,
+                s.pointNumBytes,
+                s.vectorDimension,
+                s.vectorSearchStrategy,
+                pf.fieldName.equals(fieldInfos.getSoftDeletesFieldName())));
+    pf.setFieldInfo(fi);
+    if (fi.getIndexOptions() != IndexOptions.NONE) {
+      pf.setInvertState();
+    }
+    DocValuesType dvType = fi.getDocValuesType();
+    switch (dvType) {
+      case NONE:
+        break;
+      case NUMERIC:
+        pf.docValuesWriter = new NumericDocValuesWriter(fi, bytesUsed);
+        break;
+      case BINARY:
+        pf.docValuesWriter = new BinaryDocValuesWriter(fi, bytesUsed);
+        break;
+      case SORTED:
+        pf.docValuesWriter = new SortedDocValuesWriter(fi, bytesUsed);
+        break;
+      case SORTED_NUMERIC:
+        pf.docValuesWriter = new SortedNumericDocValuesWriter(fi, bytesUsed);
+        break;
+      case SORTED_SET:
+        pf.docValuesWriter = new SortedSetDocValuesWriter(fi, bytesUsed);
+        break;
+      default:
+        throw new AssertionError("unrecognized DocValues.Type: " + dvType);
+    }
+    if (fi.getPointDimensionCount() != 0) {
+      pf.pointValuesWriter = new PointValuesWriter(byteBlockAllocator, bytesUsed, fi);
+    }
+    if (fi.getVectorDimension() != 0) {
+      pf.vectorValuesWriter = new VectorValuesWriter(fi, bytesUsed);
     }
   }
 
-  private int processField(int docID, IndexableField field, long fieldGen, int fieldCount)
-      throws IOException {
-    String fieldName = field.name();
+  /** Index each field Returns {@code true}, if we are indexing a unique field with postings */
+  private boolean processField(int docID, IndexableField field, PerField pf) throws IOException {
     IndexableFieldType fieldType = field.fieldType();
+    boolean indexedField = false;
 
-    PerField fp = null;
-
-    if (fieldType.indexOptions() == null) {
-      throw new NullPointerException(
-          "IndexOptions must not be null (field: \"" + field.name() + "\")");
-    }
-
-    // Invert indexed fields:
+    // Invert indexed fields
     if (fieldType.indexOptions() != IndexOptions.NONE) {
-      fp = getOrAddField(fieldName, fieldType, true);
-      boolean first = fp.fieldGen != fieldGen;
-      fp.invert(docID, field, first);
-
-      if (first) {
-        fields[fieldCount++] = fp;
-        fp.fieldGen = fieldGen;
+      if (pf.first) { // first time we see this field in this doc
+        pf.invert(docID, field, true);
+        pf.first = false;
+        indexedField = true;
+      } else {
+        pf.invert(docID, field, false);
       }
-    } else {
-      verifyUnIndexedFieldType(fieldName, fieldType);
     }
 
-    // Add stored fields:
+    // Add stored fields
     if (fieldType.stored()) {
-      if (fp == null) {
-        fp = getOrAddField(fieldName, fieldType, false);
-      }
       String value = field.stringValue();
       if (value != null && value.length() > IndexWriter.MAX_STORED_STRING_LENGTH) {
         throw new IllegalArgumentException(
@@ -663,7 +743,7 @@ final class IndexingChain implements Accountable {
                 + " characters) to store");
       }
       try {
-        storedFieldsConsumer.writeField(fp.fieldInfo, field);
+        storedFieldsConsumer.writeField(pf.fieldInfo, field);
       } catch (Throwable th) {
         onAbortingException(th);
         throw th;
@@ -671,30 +751,82 @@ final class IndexingChain implements Accountable {
     }
 
     DocValuesType dvType = fieldType.docValuesType();
-    if (dvType == null) {
-      throw new NullPointerException(
-          "docValuesType must not be null (field: \"" + fieldName + "\")");
-    }
     if (dvType != DocValuesType.NONE) {
-      if (fp == null) {
-        fp = getOrAddField(fieldName, fieldType, false);
-      }
-      indexDocValue(docID, fp, dvType, field);
+      indexDocValue(docID, pf, dvType, field);
     }
     if (fieldType.pointDimensionCount() != 0) {
-      if (fp == null) {
-        fp = getOrAddField(fieldName, fieldType, false);
-      }
-      indexPoint(docID, fp, field);
+      pf.pointValuesWriter.addPackedValue(docID, field.binaryValue());
     }
     if (fieldType.vectorDimension() != 0) {
-      if (fp == null) {
-        fp = getOrAddField(fieldName, fieldType, false);
+      pf.vectorValuesWriter.addValue(docID, ((VectorField) field).vectorValue());
+    }
+    return indexedField;
+  }
+
+  /**
+   * Returns a previously created {@link PerField}, absorbing the type information from {@link
+   * FieldType}, and creates a new {@link PerField} if this field name wasn't seen yet.
+   */
+  private PerField getOrAddPerField(String fieldName, IndexableFieldType fieldType) {
+    final int hashPos = fieldName.hashCode() & hashMask;
+    PerField pf = fieldHash[hashPos];
+    while (pf != null && pf.fieldName.equals(fieldName) == false) {
+      pf = pf.next;
+    }
+    if (pf == null) {
+      // first time we encounter field with this name in this segment
+      FieldSchema schema = new FieldSchema(fieldName);
+      pf =
+          new PerField(
+              fieldName,
+              indexCreatedVersionMajor,
+              schema,
+              indexWriterConfig.getSimilarity(),
+              indexWriterConfig.getInfoStream(),
+              indexWriterConfig.getAnalyzer());
+      pf.next = fieldHash[hashPos];
+      fieldHash[hashPos] = pf;
+      totalFieldCount++;
+      // At most 50% load factor:
+      if (totalFieldCount >= fieldHash.length / 2) {
+        rehash();
+      }
+      if (totalFieldCount > fields.length) {
+        PerField[] newFields =
+            new PerField
+                [ArrayUtil.oversize(totalFieldCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+        System.arraycopy(fields, 0, newFields, 0, fields.length);
+        fields = newFields;
       }
-      indexVector(docID, fp, field);
     }
+    return pf;
+  }
 
-    return fieldCount;
+  // update schema for field as seen in a particular document
+  private static void updateDocFieldSchema(
+      String fieldName, FieldSchema schema, IndexableFieldType fieldType) {
+    if (fieldType.indexOptions() != IndexOptions.NONE) {
+      schema.setIndexOptions(
+          fieldType.indexOptions(), fieldType.omitNorms(), fieldType.storeTermVectors());
+    } else {
+      // TODO: should this be checked when a fieldType is created?
+      verifyUnIndexedFieldType(fieldName, fieldType);
+    }
+    if (fieldType.docValuesType() != DocValuesType.NONE) {
+      schema.setDocValues(fieldType.docValuesType(), -1);
+    }
+    if (fieldType.pointDimensionCount() != 0) {
+      schema.setPoints(
+          fieldType.pointDimensionCount(),
+          fieldType.pointIndexDimensionCount(),
+          fieldType.pointNumBytes());
+    }
+    if (fieldType.vectorDimension() != 0) {
+      schema.setVectors(fieldType.vectorSearchStrategy(), fieldType.vectorDimension());
+    }
+    if (fieldType.getAttributes() != null && fieldType.getAttributes().isEmpty() == false) {
+      schema.updateAttributes(fieldType.getAttributes());
+    }
   }
 
   private static void verifyUnIndexedFieldType(String name, IndexableFieldType ft) {
@@ -728,33 +860,6 @@ final class IndexingChain implements Accountable {
     }
   }
 
-  /** Called from processDocument to index one field's point */
-  private void indexPoint(int docID, PerField fp, IndexableField field) {
-    int pointDimensionCount = field.fieldType().pointDimensionCount();
-    int pointIndexDimensionCount = field.fieldType().pointIndexDimensionCount();
-
-    int dimensionNumBytes = field.fieldType().pointNumBytes();
-
-    // Record dimensions for this field; this setter will throw IllegalArgExc if
-    // the dimensions were already set to something different:
-    if (fp.fieldInfo.getPointDimensionCount() == 0) {
-      fieldInfos.globalFieldNumbers.setDimensions(
-          fp.fieldInfo.number,
-          fp.fieldInfo.name,
-          pointDimensionCount,
-          pointIndexDimensionCount,
-          dimensionNumBytes);
-    }
-
-    fp.fieldInfo.setPointDimensions(
-        pointDimensionCount, pointIndexDimensionCount, dimensionNumBytes);
-
-    if (fp.pointValuesWriter == null) {
-      fp.pointValuesWriter = new PointValuesWriter(byteBlockAllocator, bytesUsed, fp.fieldInfo);
-    }
-    fp.pointValuesWriter.addPackedValue(docID, field.binaryValue());
-  }
-
   private void validateIndexSortDVType(Sort indexSort, String fieldToValidate, DocValuesType dvType)
       throws IOException {
     for (SortField sortField : indexSort.getSort()) {
@@ -850,28 +955,9 @@ final class IndexingChain implements Accountable {
   }
 
   /** Called from processDocument to index one field's doc value */
-  private void indexDocValue(int docID, PerField fp, DocValuesType dvType, IndexableField field)
-      throws IOException {
-
-    if (fp.fieldInfo.getDocValuesType() == DocValuesType.NONE) {
-      // This is the first time we are seeing this field indexed with doc values, so we
-      // now record the DV type so that any future attempt to (illegally) change
-      // the DV type of this field, will throw an IllegalArgExc:
-      if (indexWriterConfig.getIndexSort() != null) {
-        final Sort indexSort = indexWriterConfig.getIndexSort();
-        validateIndexSortDVType(indexSort, fp.fieldInfo.name, dvType);
-      }
-      fieldInfos.globalFieldNumbers.setDocValuesType(
-          fp.fieldInfo.number, fp.fieldInfo.name, dvType);
-    }
-
-    fp.fieldInfo.setDocValuesType(dvType);
-
+  private void indexDocValue(int docID, PerField fp, DocValuesType dvType, IndexableField field) {
     switch (dvType) {
       case NUMERIC:
-        if (fp.docValuesWriter == null) {
-          fp.docValuesWriter = new NumericDocValuesWriter(fp.fieldInfo, bytesUsed);
-        }
         if (field.numericValue() == null) {
           throw new IllegalArgumentException(
               "field=\"" + fp.fieldInfo.name + "\": null value not allowed");
@@ -881,31 +967,19 @@ final class IndexingChain implements Accountable {
         break;
 
       case BINARY:
-        if (fp.docValuesWriter == null) {
-          fp.docValuesWriter = new BinaryDocValuesWriter(fp.fieldInfo, bytesUsed);
-        }
         ((BinaryDocValuesWriter) fp.docValuesWriter).addValue(docID, field.binaryValue());
         break;
 
       case SORTED:
-        if (fp.docValuesWriter == null) {
-          fp.docValuesWriter = new SortedDocValuesWriter(fp.fieldInfo, bytesUsed);
-        }
         ((SortedDocValuesWriter) fp.docValuesWriter).addValue(docID, field.binaryValue());
         break;
 
       case SORTED_NUMERIC:
-        if (fp.docValuesWriter == null) {
-          fp.docValuesWriter = new SortedNumericDocValuesWriter(fp.fieldInfo, bytesUsed);
-        }
         ((SortedNumericDocValuesWriter) fp.docValuesWriter)
             .addValue(docID, field.numericValue().longValue());
         break;
 
       case SORTED_SET:
-        if (fp.docValuesWriter == null) {
-          fp.docValuesWriter = new SortedSetDocValuesWriter(fp.fieldInfo, bytesUsed);
-        }
         ((SortedSetDocValuesWriter) fp.docValuesWriter).addValue(docID, field.binaryValue());
         break;
 
@@ -915,103 +989,16 @@ final class IndexingChain implements Accountable {
     }
   }
 
-  /** Called from processDocument to index one field's vector value */
-  private void indexVector(int docID, PerField fp, IndexableField field) {
-    int dimension = field.fieldType().vectorDimension();
-    VectorValues.SearchStrategy searchStrategy = field.fieldType().vectorSearchStrategy();
-
-    // Record dimensions and distance function for this field; this setter will throw IllegalArgExc
-    // if
-    // the dimensions or distance function were already set to something different:
-    if (fp.fieldInfo.getVectorDimension() == 0) {
-      fieldInfos.globalFieldNumbers.setVectorDimensionsAndSearchStrategy(
-          fp.fieldInfo.number, fp.fieldInfo.name, dimension, searchStrategy);
-    }
-    fp.fieldInfo.setVectorDimensionAndSearchStrategy(dimension, searchStrategy);
-
-    if (fp.vectorValuesWriter == null) {
-      fp.vectorValuesWriter = new VectorValuesWriter(fp.fieldInfo, bytesUsed);
-    }
-    fp.vectorValuesWriter.addValue(docID, ((VectorField) field).vectorValue());
-  }
-
   /** Returns a previously created {@link PerField}, or null if this field name wasn't seen yet. */
   private PerField getPerField(String name) {
     final int hashPos = name.hashCode() & hashMask;
     PerField fp = fieldHash[hashPos];
-    while (fp != null && !fp.fieldInfo.name.equals(name)) {
+    while (fp != null && !fp.fieldName.equals(name)) {
       fp = fp.next;
     }
     return fp;
   }
 
-  /**
-   * Returns a previously created {@link PerField}, absorbing the type information from {@link
-   * FieldType}, and creates a new {@link PerField} if this field name wasn't seen yet.
-   */
-  private PerField getOrAddField(String name, IndexableFieldType fieldType, boolean invert) {
-
-    // Make sure we have a PerField allocated
-    final int hashPos = name.hashCode() & hashMask;
-    PerField fp = fieldHash[hashPos];
-    while (fp != null && !fp.fieldInfo.name.equals(name)) {
-      fp = fp.next;
-    }
-
-    if (fp == null) {
-      // First time we are seeing this field in this segment
-
-      FieldInfo fi = fieldInfos.getOrAdd(name);
-      initIndexOptions(fi, fieldType.indexOptions());
-      Map<String, String> attributes = fieldType.getAttributes();
-      if (attributes != null) {
-        attributes.forEach((k, v) -> fi.putAttribute(k, v));
-      }
-
-      fp =
-          new PerField(
-              indexCreatedVersionMajor,
-              fi,
-              invert,
-              indexWriterConfig.getSimilarity(),
-              indexWriterConfig.getInfoStream(),
-              indexWriterConfig.getAnalyzer());
-      fp.next = fieldHash[hashPos];
-      fieldHash[hashPos] = fp;
-      totalFieldCount++;
-
-      // At most 50% load factor:
-      if (totalFieldCount >= fieldHash.length / 2) {
-        rehash();
-      }
-
-      if (totalFieldCount > fields.length) {
-        PerField[] newFields =
-            new PerField
-                [ArrayUtil.oversize(totalFieldCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
-        System.arraycopy(fields, 0, newFields, 0, fields.length);
-        fields = newFields;
-      }
-
-    } else if (invert && fp.invertState == null) {
-      initIndexOptions(fp.fieldInfo, fieldType.indexOptions());
-      fp.setInvertState();
-    }
-
-    return fp;
-  }
-
-  private void initIndexOptions(FieldInfo info, IndexOptions indexOptions) {
-    // Messy: must set this here because e.g. FreqProxTermsWriterPerField looks at the initial
-    // IndexOptions to decide what arrays it must create).
-    assert info.getIndexOptions() == IndexOptions.NONE;
-    // This is the first time we are seeing this field indexed, so we now
-    // record the index options so that any future attempt to (illegally)
-    // change the index options of this field, will throw an IllegalArgExc:
-    fieldInfos.globalFieldNumbers.setIndexOptions(info.number, info.name, indexOptions);
-    info.setIndexOptions(indexOptions);
-  }
-
   @Override
   public long ramBytesUsed() {
     return bytesUsed.get()
@@ -1026,9 +1013,10 @@ final class IndexingChain implements Accountable {
 
   /** NOTE: not static: accesses at least docState, termsHash. */
   private final class PerField implements Comparable<PerField> {
-
+    final String fieldName;
     final int indexCreatedVersionMajor;
-    final FieldInfo fieldInfo;
+    final FieldSchema schema;
+    FieldInfo fieldInfo;
     final Similarity similarity;
 
     FieldInvertState invertState;
@@ -1057,22 +1045,31 @@ final class IndexingChain implements Accountable {
     TokenStream tokenStream;
     private final InfoStream infoStream;
     private final Analyzer analyzer;
+    private boolean first; // first in a document
 
     PerField(
+        String fieldName,
         int indexCreatedVersionMajor,
-        FieldInfo fieldInfo,
-        boolean invert,
+        FieldSchema schema,
         Similarity similarity,
         InfoStream infoStream,
         Analyzer analyzer) {
+      this.fieldName = fieldName;
       this.indexCreatedVersionMajor = indexCreatedVersionMajor;
-      this.fieldInfo = fieldInfo;
+      this.schema = schema;
       this.similarity = similarity;
       this.infoStream = infoStream;
       this.analyzer = analyzer;
-      if (invert) {
-        setInvertState();
-      }
+    }
+
+    void reset(int docId) {
+      first = true;
+      schema.reset(docId);
+    }
+
+    void setFieldInfo(FieldInfo fieldInfo) {
+      assert this.fieldInfo == null;
+      this.fieldInfo = fieldInfo;
     }
 
     void setInvertState() {
@@ -1083,14 +1080,17 @@ final class IndexingChain implements Accountable {
       if (fieldInfo.omitsNorms() == false) {
         assert norms == null;
         // Even if no documents actually succeed in setting a norm, we still write norms for this
-        // segment:
+        // segment
         norms = new NormValuesWriter(fieldInfo, bytesUsed);
       }
+      if (fieldInfo.hasVectors()) {
+        termVectorsWriter.setHasVectors();
+      }
     }
 
     @Override
     public int compareTo(PerField other) {
-      return this.fieldInfo.name.compareTo(other.fieldInfo.name);
+      return this.fieldName.compareTo(other.fieldName);
     }
 
     public void finish(int docID) throws IOException {
@@ -1110,7 +1110,6 @@ final class IndexingChain implements Accountable {
         }
         norms.addValue(docID, normValue);
       }
-
       termsHashPerField.finish();
     }
 
@@ -1120,22 +1119,11 @@ final class IndexingChain implements Accountable {
      */
     public void invert(int docID, IndexableField field, boolean first) throws IOException {
       if (first) {
-        // First time we're seeing this field (indexed) in
-        // this document:
+        // First time we're seeing this field (indexed) in this document
         invertState.reset();
       }
 
-      IndexableFieldType fieldType = field.fieldType();
-
-      IndexOptions indexOptions = fieldType.indexOptions();
-      fieldInfo.setIndexOptions(indexOptions);
-
-      if (fieldType.omitNorms()) {
-        fieldInfo.setOmitsNorms();
-      }
-
-      final boolean analyzed = fieldType.tokenized() && analyzer != null;
-
+      final boolean analyzed = field.fieldType().tokenized() && analyzer != null;
       /*
        * To assist people in tracking down problems in analysis components, we wish to write the field name to the infostream
        * when we fail. We expect some caller to eventually deal with the real exception, so we don't want any 'catch' clauses,
@@ -1314,4 +1302,118 @@ final class IndexingChain implements Accountable {
       bytesUsed.addAndGet(-(length * (IntBlockPool.INT_BLOCK_SIZE * Integer.BYTES)));
     }
   }
+
+  /**
+   * A schema of the field in the current document. With every new document this schema is reset. As
+   * the document fields are processed, we update the schema with options encountered in this
+   * document. Once the processing for the document is done, we compare the built schema of the
+   * current document with the corresponding FieldInfo (FieldInfo is built on a first document in
+   * the segment where we encounter this field). If there is inconsistency, we raise an error. This
+   * ensures that a field has the same data structures across all documents.
+   */
+  private static final class FieldSchema {
+    private final String name;
+    private int docID = 0;
+    private final Map<String, String> attributes = new HashMap<>();
+    private boolean omitNorms = false;
+    private boolean storeTermVector = false;
+    private IndexOptions indexOptions = IndexOptions.NONE;
+    private long dvGen = -1;
+    private DocValuesType docValuesType = DocValuesType.NONE;
+    private int pointDimensionCount = 0;
+    private int pointIndexDimensionCount = 0;
+    private int pointNumBytes = 0;
+    private int vectorDimension = 0;
+    private VectorValues.SearchStrategy vectorSearchStrategy = VectorValues.SearchStrategy.NONE;
+
+    private static String errMsg =
+        "Inconsistency of field data structures across documents for field ";
+
+    FieldSchema(String name) {
+      this.name = name;
+    }
+
+    private void assertSame(boolean same) {
+      if (same == false) {
+        throw new IllegalArgumentException(errMsg + "[" + name + "] of doc [" + docID + "].");
+      }
+    }
+
+    void updateAttributes(Map<String, String> attrs) {
+      attrs.forEach((k, v) -> this.attributes.put(k, v));
+    }
+
+    void setIndexOptions(
+        IndexOptions newIndexOptions, boolean newOmitNorms, boolean newStoreTermVector) {
+      if (indexOptions == IndexOptions.NONE) {
+        indexOptions = newIndexOptions;
+        omitNorms = newOmitNorms;
+        storeTermVector = newStoreTermVector;
+      } else {
+        assertSame(
+            indexOptions == newIndexOptions
+                && omitNorms == newOmitNorms
+                && storeTermVector == newStoreTermVector);
+      }
+    }
+
+    void setDocValues(DocValuesType newDocValuesType, long newDvGen) {
+      if (docValuesType == DocValuesType.NONE) {
+        this.docValuesType = newDocValuesType;
+        this.dvGen = newDvGen;
+      } else {
+        assertSame(docValuesType == newDocValuesType && dvGen == newDvGen);
+      }
+    }
+
+    void setPoints(int dimensionCount, int indexDimensionCount, int numBytes) {
+      if (pointIndexDimensionCount == 0) {
+        pointDimensionCount = dimensionCount;
+        pointIndexDimensionCount = indexDimensionCount;
+        pointNumBytes = numBytes;
+      } else {
+        assertSame(
+            pointDimensionCount == dimensionCount
+                && pointIndexDimensionCount == indexDimensionCount
+                && pointNumBytes == numBytes);
+      }
+    }
+
+    void setVectors(VectorValues.SearchStrategy searchStrategy, int dimension) {
+      if (vectorSearchStrategy == VectorValues.SearchStrategy.NONE) {
+        this.vectorDimension = dimension;
+        this.vectorSearchStrategy = searchStrategy;
+      } else {
+        assertSame(vectorSearchStrategy == searchStrategy && vectorDimension == dimension);
+      }
+    }
+
+    void reset(int doc) {
+      docID = doc;
+      omitNorms = false;
+      storeTermVector = false;
+      indexOptions = IndexOptions.NONE;
+      dvGen = -1;
+      docValuesType = DocValuesType.NONE;
+      pointDimensionCount = 0;
+      pointIndexDimensionCount = 0;
+      pointNumBytes = 0;
+      vectorDimension = 0;
+      vectorSearchStrategy = VectorValues.SearchStrategy.NONE;
+    }
+
+    void assertSameSchema(FieldInfo fi) {
+      assertSame(
+          indexOptions == fi.getIndexOptions()
+              && omitNorms == fi.omitsNorms()
+              && storeTermVector == fi.hasVectors()
+              && docValuesType == fi.getDocValuesType()
+              && dvGen == fi.getDocValuesGen()
+              && pointDimensionCount == fi.getPointDimensionCount()
+              && pointIndexDimensionCount == fi.getPointIndexDimensionCount()
+              && pointNumBytes == fi.getPointNumBytes()
+              && vectorDimension == fi.getVectorDimension()
+              && vectorSearchStrategy == fi.getVectorSearchStrategy());
+    }
+  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
index e04a992..52d83ec 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
@@ -75,6 +75,15 @@ class PointValuesWriter {
     numPoints++;
   }
 
+  /**
+   * Get number of buffered documents
+   *
+   * @return number of buffered documents
+   */
+  public int getNumDocs() {
+    return numDocs;
+  }
+
   public void flush(SegmentWriteState state, Sorter.DocMap sortMap, PointsWriter writer)
       throws IOException {
     PointValues points =
diff --git a/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java b/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
index b917be5..312e5e0 100644
--- a/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
+++ b/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
@@ -602,20 +602,19 @@ final class ReadersAndUpdates {
         }
 
         // create new fields with the right DV type
-        FieldInfos.Builder builder = new FieldInfos.Builder(fieldNumbers);
         for (List<DocValuesFieldUpdates> updates : pendingDVUpdates.values()) {
           DocValuesFieldUpdates update = updates.get(0);
-
           if (byName.containsKey(update.field)) {
             // the field already exists in this segment
             FieldInfo fi = byName.get(update.field);
-            fi.setDocValuesType(update.type);
+            assert fi.getDocValuesType() == update.type;
           } else {
             // the field is not present in this segment so we clone the global field
             // (which is guaranteed to exist) and remaps its field number locally.
-            assert fieldNumbers.contains(update.field, update.type);
-            FieldInfo fi = cloneFieldInfo(builder.getOrAdd(update.field), ++maxFieldNumber);
-            fi.setDocValuesType(update.type);
+            FieldInfo fi =
+                fieldNumbers.constructFieldInfo(update.field, update.type, maxFieldNumber + 1);
+            assert fi != null;
+            maxFieldNumber++;
             byName.put(fi.name, fi);
           }
         }
diff --git a/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java b/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java
index 55301c4..c6ffab8 100644
--- a/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java
@@ -48,7 +48,7 @@ class TermVectorsConsumer extends TermsHash {
 
   final ByteSliceReader vectorSliceReaderOff = new ByteSliceReader();
 
-  boolean hasVectors;
+  private boolean hasVectors;
   private int numVectorFields;
   int lastDocID;
   private TermVectorsConsumerPerField[] perFields = new TermVectorsConsumerPerField[1];
@@ -109,6 +109,10 @@ class TermVectorsConsumer extends TermsHash {
     }
   }
 
+  void setHasVectors() {
+    hasVectors = true;
+  }
+
   @Override
   void finishDocument(int docID) throws IOException {
 
diff --git a/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java b/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java
index 40c19db..a4c2448 100644
--- a/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java
+++ b/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java
@@ -149,9 +149,6 @@ final class TermVectorsConsumerPerField extends TermsHashPerField {
       doVectors = field.fieldType().storeTermVectors();
 
       if (doVectors) {
-
-        termsWriter.hasVectors = true;
-
         doVectorPositions = field.fieldType().storeTermVectorPositions();
 
         // Somewhat confusingly, unlike postings, you are
diff --git a/lucene/core/src/test/org/apache/lucene/document/TestPerFieldConsistency.java b/lucene/core/src/test/org/apache/lucene/document/TestPerFieldConsistency.java
new file mode 100644
index 0000000..54088fc
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/document/TestPerFieldConsistency.java
@@ -0,0 +1,224 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.document;
+
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomDouble;
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomFloat;
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomInt;
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomIntBetween;
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomLong;
+
+import com.carrotsearch.randomizedtesting.generators.RandomPicks;
+import java.io.IOException;
+import java.util.Random;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestPerFieldConsistency extends LuceneTestCase {
+
+  private static Field randomIndexedField(Random random, String fieldName) {
+    FieldType fieldType = new FieldType();
+    IndexOptions indexOptions = RandomPicks.randomFrom(random, IndexOptions.values());
+    while (indexOptions == IndexOptions.NONE) {
+      indexOptions = RandomPicks.randomFrom(random, IndexOptions.values());
+    }
+    fieldType.setIndexOptions(indexOptions);
+    fieldType.setStoreTermVectors(random.nextBoolean());
+    if (fieldType.storeTermVectors()) {
+      fieldType.setStoreTermVectorPositions(random.nextBoolean());
+      if (fieldType.storeTermVectorPositions()) {
+        fieldType.setStoreTermVectorPayloads(random.nextBoolean());
+        fieldType.setStoreTermVectorOffsets(random.nextBoolean());
+      }
+    }
+    fieldType.setOmitNorms(random.nextBoolean());
+    fieldType.setStored(random.nextBoolean());
+    fieldType.freeze();
+
+    return new Field(fieldName, "randomValue", fieldType);
+  }
+
+  private static Field randomPointField(Random random, String fieldName) {
+    switch (random.nextInt(4)) {
+      case 0:
+        return new LongPoint(fieldName, randomLong());
+      case 1:
+        return new IntPoint(fieldName, randomInt());
+      case 2:
+        return new DoublePoint(fieldName, randomDouble());
+      default:
+        return new FloatPoint(fieldName, randomFloat());
+    }
+  }
+
+  private static Field randomDocValuesField(Random random, String fieldName) {
+    switch (random.nextInt(4)) {
+      case 0:
+        return new BinaryDocValuesField(fieldName, new BytesRef("randomValue"));
+      case 1:
+        return new NumericDocValuesField(fieldName, randomLong());
+      case 2:
+        return new DoubleDocValuesField(fieldName, randomDouble());
+      default:
+        return new SortedSetDocValuesField(fieldName, new BytesRef("randomValue"));
+    }
+  }
+
+  private static Field randomVectorField(Random random, String fieldName) {
+    VectorValues.SearchStrategy searchStrategy =
+        RandomPicks.randomFrom(random, VectorValues.SearchStrategy.values());
+    while (searchStrategy == VectorValues.SearchStrategy.NONE) {
+      searchStrategy = RandomPicks.randomFrom(random, VectorValues.SearchStrategy.values());
+    }
+    float[] values = new float[randomIntBetween(1, 10)];
+    for (int i = 0; i < values.length; i++) {
+      values[i] = randomFloat();
+    }
+    return new VectorField(fieldName, values, searchStrategy);
+  }
+
+  private static Field[] randomFieldsWithTheSameName(String fieldName) {
+    final Field textField = randomIndexedField(random(), fieldName);
+    final Field docValuesField = randomDocValuesField(random(), fieldName);
+    final Field pointField = randomPointField(random(), fieldName);
+    final Field vectorField = randomVectorField(random(), fieldName);
+    return new Field[] {textField, docValuesField, pointField, vectorField};
+  }
+
+  private static void doTestDocWithMissingSchemaOptionsThrowsError(
+      Field[] fields, int missing, IndexWriter writer, String errorMsg) {
+    final Document doc = new Document();
+    for (int i = 0; i < fields.length; i++) {
+      if (i != missing) {
+        doc.add(fields[i]);
+      }
+    }
+    IllegalArgumentException exception =
+        expectThrows(IllegalArgumentException.class, () -> writer.addDocument(doc));
+    assertTrue(exception.getMessage().contains(errorMsg));
+  }
+
+  private static void doTestDocWithExtraSchemaOptionsThrowsError(
+      Field existing, Field extra, IndexWriter writer, String errorMsg) {
+    Document doc = new Document();
+    doc.add(existing);
+    doc.add(extra);
+    IllegalArgumentException exception =
+        expectThrows(IllegalArgumentException.class, () -> writer.addDocument(doc));
+    assertTrue(exception.getMessage().contains(errorMsg));
+  }
+
+  public void testDocWithMissingSchemaOptionsThrowsError() throws IOException {
+    try (Directory dir = newDirectory();
+        IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig()); ) {
+      final Field[] fields = randomFieldsWithTheSameName("myfield");
+      final Document doc0 = new Document();
+      for (Field field : fields) {
+        doc0.add(field);
+      }
+      writer.addDocument(doc0);
+
+      // the same segment: indexing a doc with a missing field throws error
+      int numNotIndexedDocs = 0;
+      for (int missingFieldIdx = 0; missingFieldIdx < fields.length; missingFieldIdx++) {
+        numNotIndexedDocs++;
+        doTestDocWithMissingSchemaOptionsThrowsError(
+            fields,
+            missingFieldIdx,
+            writer,
+            "Inconsistency of field data structures across documents for field [myfield] of doc ["
+                + numNotIndexedDocs
+                + "].");
+      }
+      writer.flush();
+      try (IndexReader reader = DirectoryReader.open(writer)) {
+        assertEquals(1, reader.leaves().get(0).reader().numDocs());
+        assertEquals(numNotIndexedDocs, reader.leaves().get(0).reader().numDeletedDocs());
+      }
+
+      // diff segment, same index: indexing a doc with a missing field throws error
+      numNotIndexedDocs = 0;
+      for (int missingFieldIdx = 0; missingFieldIdx < fields.length; missingFieldIdx++) {
+        numNotIndexedDocs++;
+        doTestDocWithMissingSchemaOptionsThrowsError(
+            fields, missingFieldIdx, writer, "cannot change field \"myfield\" from ");
+      }
+      writer.addDocument(doc0); // add document with correct data structures
+      writer.flush();
+      try (IndexReader reader = DirectoryReader.open(writer)) {
+        assertEquals(1, reader.leaves().get(1).reader().numDocs());
+        assertEquals(numNotIndexedDocs, reader.leaves().get(1).reader().numDeletedDocs());
+      }
+    }
+  }
+
+  public void testDocWithExtraSchemaOptionsThrowsError() throws IOException {
+    try (Directory dir = newDirectory();
+        IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig())) {
+      final Field[] fields = randomFieldsWithTheSameName("myfield");
+      final Document doc0 = new Document();
+      int existingFieldIdx = randomIntBetween(0, fields.length - 1);
+      doc0.add(fields[existingFieldIdx]);
+      writer.addDocument(doc0);
+
+      // the same segment: indexing a field with extra field indexing options returns error
+      int numNotIndexedDocs = 0;
+      for (int extraFieldIndex = 0; extraFieldIndex < fields.length; extraFieldIndex++) {
+        if (extraFieldIndex == existingFieldIdx) continue;
+        numNotIndexedDocs++;
+        doTestDocWithExtraSchemaOptionsThrowsError(
+            fields[existingFieldIdx],
+            fields[extraFieldIndex],
+            writer,
+            "Inconsistency of field data structures across documents for field [myfield] of doc ["
+                + numNotIndexedDocs
+                + "].");
+      }
+      writer.flush();
+      try (IndexReader reader = DirectoryReader.open(writer)) {
+        assertEquals(1, reader.leaves().get(0).reader().numDocs());
+        assertEquals(numNotIndexedDocs, reader.leaves().get(0).reader().numDeletedDocs());
+      }
+
+      // diff segment, same index: indexing a field with extra field indexing options returns error
+      numNotIndexedDocs = 0;
+      for (int extraFieldIndex = 0; extraFieldIndex < fields.length; extraFieldIndex++) {
+        if (extraFieldIndex == existingFieldIdx) continue;
+        numNotIndexedDocs++;
+        doTestDocWithExtraSchemaOptionsThrowsError(
+            fields[existingFieldIdx],
+            fields[extraFieldIndex],
+            writer,
+            "cannot change field \"myfield\" from ");
+      }
+      writer.addDocument(doc0); // add document with correct data structures
+      writer.flush();
+      try (IndexReader reader = DirectoryReader.open(writer)) {
+        assertEquals(1, reader.leaves().get(1).reader().numDocs());
+        assertEquals(numNotIndexedDocs, reader.leaves().get(1).reader().numDeletedDocs());
+      }
+    }
+  }
+}
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestAtomicUpdate.java b/lucene/core/src/test/org/apache/lucene/index/TestAtomicUpdate.java
index 54dfd05..f4e3a93 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestAtomicUpdate.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestAtomicUpdate.java
@@ -24,8 +24,6 @@ import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.IntPoint;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.document.TextField;
 import org.apache.lucene.store.ByteBuffersDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.MockDirectoryWrapper;
@@ -71,9 +69,9 @@ public class TestAtomicUpdate extends LuceneTestCase {
       // Update all 100 docs...
       for (int i = 0; i < 100; i++) {
         Document d = new Document();
-        d.add(new StringField("id", Integer.toString(i), Field.Store.YES));
+        d.add(newStringField("id", Integer.toString(i), Field.Store.YES));
         d.add(
-            new TextField(
+            newTextField(
                 "contents", English.intToEnglish(i + 10 * currentIteration), Field.Store.NO));
         d.add(new IntPoint("doc", i));
         d.add(new IntPoint("doc2d", i, i));
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java b/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java
index 74658ad..af37227 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java
@@ -953,27 +953,41 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
     writer.commit();
 
     // second segment with no BDV
-    doc = new Document();
-    doc.add(new StringField("id", "doc1", Store.NO));
-    doc.add(new StringField("bdv", "mock-value", Store.NO));
-    writer.addDocument(doc);
-    writer.commit();
+    Document doc2 = new Document();
+    doc2.add(new StringField("id", "doc1", Store.NO));
+    doc2.add(new StringField("bdv", "mock-value", Store.NO));
+    IllegalArgumentException exception =
+        expectThrows(IllegalArgumentException.class, () -> writer.addDocument(doc2));
+    String expectedErrMsg =
+        "cannot change field \"bdv\" from doc values type=BINARY to inconsistent doc values type=NONE";
+    assertEquals(expectedErrMsg, exception.getMessage());
+
+    doc2.add(new BinaryDocValuesField("bdv", toBytes(10L)));
+    writer.addDocument(doc2);
+
+    // update doc values of bdv field in the second segment
+    exception =
+        expectThrows(
+            IllegalArgumentException.class,
+            () -> writer.updateBinaryDocValue(new Term("id", "doc1"), "bdv", toBytes(5L)));
+    expectedErrMsg =
+        "Can't update [BINARY] doc values; the field [bdv] must be doc values only field, but is also indexed with postings.";
+    assertEquals(expectedErrMsg, exception.getMessage());
 
-    // update document in the second segment
-    writer.updateBinaryDocValue(new Term("id", "doc1"), "bdv", toBytes(5L));
+    writer.commit();
     writer.close();
 
     DirectoryReader reader = DirectoryReader.open(dir);
-    for (LeafReaderContext context : reader.leaves()) {
-      LeafReader r = context.reader();
-      BinaryDocValues bdv = r.getBinaryDocValues("bdv");
-      for (int i = 0; i < r.maxDoc(); i++) {
-        assertEquals(i, bdv.nextDoc());
-        assertEquals(5L, getValue(bdv));
-      }
-    }
-    reader.close();
+    LeafReader r1 = reader.leaves().get(0).reader();
+    BinaryDocValues bdv1 = r1.getBinaryDocValues("bdv");
+    assertEquals(0, bdv1.nextDoc());
+    assertEquals(5L, getValue(bdv1));
+    LeafReader r2 = reader.leaves().get(1).reader();
+    BinaryDocValues bdv2 = r2.getBinaryDocValues("bdv");
+    assertEquals(1, bdv2.nextDoc());
+    assertEquals(10L, getValue(bdv2));
 
+    reader.close();
     dir.close();
   }
 
@@ -989,13 +1003,20 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
     doc.add(new BinaryDocValuesField("f", toBytes(5L)));
     writer.addDocument(doc);
     writer.commit();
-    writer.updateBinaryDocValue(new Term("f", "mock-value"), "f", toBytes(17L));
+
+    IllegalArgumentException exception =
+        expectThrows(
+            IllegalArgumentException.class,
+            () -> writer.updateBinaryDocValue(new Term("f", "mock-value"), "f", toBytes(17L)));
+    String expectedErrMsg =
+        "Can't update [BINARY] doc values; the field [f] must be doc values only field, but is also indexed with postings.";
+    assertEquals(expectedErrMsg, exception.getMessage());
     writer.close();
 
     DirectoryReader r = DirectoryReader.open(dir);
     BinaryDocValues bdv = r.leaves().get(0).reader().getBinaryDocValues("f");
     assertEquals(0, bdv.nextDoc());
-    assertEquals(17, getValue(bdv));
+    assertEquals(5, getValue(bdv));
     r.close();
 
     dir.close();
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java b/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java
index eefe18a..77662ac 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java
@@ -90,14 +90,30 @@ public class TestCodecs extends LuceneTestCase {
       this.omitTF = omitTF;
       this.storePayloads = storePayloads;
       // TODO: change this test to use all three
-      fieldInfo = fieldInfos.getOrAdd(name);
-      if (omitTF) {
-        fieldInfo.setIndexOptions(IndexOptions.DOCS);
+      FieldInfo fieldInfo0 = fieldInfos.fieldInfo(name);
+      if (fieldInfo0 != null) {
+        fieldInfo = fieldInfo0;
       } else {
-        fieldInfo.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
-      }
-      if (storePayloads) {
-        fieldInfo.setStorePayloads();
+        IndexOptions indexOptions =
+            omitTF ? IndexOptions.DOCS : IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
+        fieldInfo =
+            fieldInfos.add(
+                new FieldInfo(
+                    name,
+                    -1,
+                    false,
+                    false,
+                    storePayloads,
+                    indexOptions,
+                    DocValuesType.NONE,
+                    -1,
+                    new HashMap<>(),
+                    0,
+                    0,
+                    0,
+                    0,
+                    VectorValues.SearchStrategy.NONE,
+                    false));
       }
       this.terms = terms;
       for (int i = 0; i < terms.length; i++) terms[i].field = this;
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java b/lucene/core/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java
index 406a4b9..8d52ed7 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java
@@ -57,10 +57,8 @@ public class TestConsistentFieldNumbers extends LuceneTestCase {
       }
 
       Document d2 = new Document();
-      FieldType customType2 = new FieldType(TextField.TYPE_STORED);
-      customType2.setStoreTermVectors(true);
       d2.add(new TextField("f2", "second field", Field.Store.NO));
-      d2.add(new Field("f1", "first field", customType2));
+      d2.add(new TextField("f1", "first field", Field.Store.YES));
       d2.add(new TextField("f3", "third field", Field.Store.NO));
       d2.add(new TextField("f4", "fourth field", Field.Store.NO));
       writer.addDocument(d2);
@@ -121,10 +119,8 @@ public class TestConsistentFieldNumbers extends LuceneTestCase {
                 .setMergePolicy(NoMergePolicy.INSTANCE));
 
     Document d2 = new Document();
-    FieldType customType2 = new FieldType(TextField.TYPE_STORED);
-    customType2.setStoreTermVectors(true);
     d2.add(new TextField("f2", "second field", Field.Store.YES));
-    d2.add(new Field("f1", "first field", customType2));
+    d2.add(new TextField("f1", "first field", Field.Store.YES));
     d2.add(new TextField("f3", "third field", Field.Store.YES));
     d2.add(new TextField("f4", "fourth field", Field.Store.YES));
     writer.addDocument(d2);
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java b/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java
index 249e195..5c981c4 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java
@@ -846,29 +846,6 @@ public class TestDocValuesIndexing extends LuceneTestCase {
     dir.close();
   }
 
-  public void testDocsWithField() throws Exception {
-    Directory dir = newDirectory();
-    IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
-    IndexWriter writer = new IndexWriter(dir, conf);
-    Document doc = new Document();
-    doc.add(new NumericDocValuesField("dv", 0L));
-    writer.addDocument(doc);
-
-    doc = new Document();
-    doc.add(new TextField("dv", "some text", Field.Store.NO));
-    doc.add(new NumericDocValuesField("dv", 0L));
-    writer.addDocument(doc);
-
-    DirectoryReader r = writer.getReader();
-    writer.close();
-
-    LeafReader subR = r.leaves().get(0).reader();
-    assertEquals(2, subR.numDocs());
-
-    r.close();
-    dir.close();
-  }
-
   public void testSameFieldNameForPostingAndDocValue() throws Exception {
     // LUCENE-5192: FieldInfos.Builder neglected to update
     // globalFieldNumbers.docValuesType map if the field existed, resulting in
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestFieldInfos.java b/lucene/core/src/test/org/apache/lucene/index/TestFieldInfos.java
index b607744..a6dc813 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestFieldInfos.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestFieldInfos.java
@@ -20,6 +20,7 @@ import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.CoreMatchers.sameInstance;
 
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.Iterator;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
@@ -149,6 +150,55 @@ public class TestFieldInfos extends LuceneTestCase {
     dir.close();
   }
 
+  public void testFieldAttributesSingleSegment() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriter writer =
+        new IndexWriter(
+            dir,
+            newIndexWriterConfig(new MockAnalyzer(random()))
+                .setMergePolicy(NoMergePolicy.INSTANCE));
+
+    Document d1 = new Document();
+    FieldType type1 = new FieldType();
+    type1.setStored(true);
+    type1.putAttribute("att1", "attdoc1");
+    d1.add(new Field("f1", "v1", type1));
+    // add field with the same name and an extra attribute
+    type1.putAttribute("att2", "attdoc1");
+    d1.add(new Field("f1", "v1", type1));
+    writer.addDocument(d1);
+
+    Document d2 = new Document();
+    type1.putAttribute("att1", "attdoc2");
+    type1.putAttribute("att2", "attdoc2");
+    type1.putAttribute("att3", "attdoc2");
+    FieldType type2 = new FieldType();
+    type2.setStored(true);
+    type2.putAttribute("att4", "attdoc2");
+    d2.add(new Field("f1", "v2", type1));
+    d2.add(new Field("f2", "v2", type2));
+    writer.addDocument(d2);
+    writer.commit();
+
+    IndexReader reader = writer.getReader();
+    FieldInfos fis = FieldInfos.getMergedFieldInfos(reader);
+
+    // test that attributes for f1 are introduced by d1,
+    // and not modified by d2
+    FieldInfo fi1 = fis.fieldInfo("f1");
+    assertEquals("attdoc1", fi1.getAttribute("att1"));
+    assertEquals("attdoc1", fi1.getAttribute("att2"));
+    assertEquals(null, fi1.getAttribute("att3"));
+
+    // test that attributes for f2 are introduced by d2
+    FieldInfo fi2 = fis.fieldInfo("f2");
+    assertEquals("attdoc2", fi2.getAttribute("att4"));
+
+    reader.close();
+    writer.close();
+    dir.close();
+  }
+
   public void testMergedFieldInfos_empty() throws IOException {
     Directory dir = newDirectory();
     IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
@@ -196,44 +246,62 @@ public class TestFieldInfos extends LuceneTestCase {
     FieldInfos.FieldNumbers fieldNumbers = new FieldInfos.FieldNumbers("softDeletes");
     for (int i = 0; i < 10; i++) {
       fieldNumbers.addOrGet(
-          "field" + i,
-          -1,
-          IndexOptions.NONE,
-          DocValuesType.NONE,
-          0,
-          0,
-          0,
-          0,
-          VectorValues.SearchStrategy.NONE,
-          false);
+          new FieldInfo(
+              "field" + i,
+              -1,
+              false,
+              false,
+              false,
+              IndexOptions.NONE,
+              DocValuesType.NONE,
+              -1,
+              new HashMap<>(),
+              0,
+              0,
+              0,
+              0,
+              VectorValues.SearchStrategy.NONE,
+              false));
     }
     int idx =
         fieldNumbers.addOrGet(
-            "EleventhField",
-            -1,
-            IndexOptions.NONE,
-            DocValuesType.NONE,
-            0,
-            0,
-            0,
-            0,
-            VectorValues.SearchStrategy.NONE,
-            false);
+            new FieldInfo(
+                "EleventhField",
+                -1,
+                false,
+                false,
+                false,
+                IndexOptions.NONE,
+                DocValuesType.NONE,
+                -1,
+                new HashMap<>(),
+                0,
+                0,
+                0,
+                0,
+                VectorValues.SearchStrategy.NONE,
+                false));
     assertEquals("Field numbers 0 through 9 were allocated", 10, idx);
 
     fieldNumbers.clear();
     idx =
         fieldNumbers.addOrGet(
-            "PostClearField",
-            -1,
-            IndexOptions.NONE,
-            DocValuesType.NONE,
-            0,
-            0,
-            0,
-            0,
-            VectorValues.SearchStrategy.NONE,
-            false);
+            new FieldInfo(
+                "PostClearField",
+                -1,
+                false,
+                false,
+                false,
+                IndexOptions.NONE,
+                DocValuesType.NONE,
+                -1,
+                new HashMap<>(),
+                0,
+                0,
+                0,
+                0,
+                VectorValues.SearchStrategy.NONE,
+                false));
     assertEquals("Field numbers should reset after clear()", 0, idx);
   }
 }
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java b/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java
index ea4dacd..2ce1cec 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java
@@ -43,16 +43,28 @@ public class TestFieldsReader extends LuceneTestCase {
   @BeforeClass
   public static void beforeClass() throws Exception {
     testDoc = new Document();
-    fieldInfos = new FieldInfos.Builder(new FieldInfos.FieldNumbers(null));
+    final String softDeletesFieldName = null;
+    fieldInfos = new FieldInfos.Builder(new FieldInfos.FieldNumbers(softDeletesFieldName));
     DocHelper.setupDoc(testDoc);
     for (IndexableField field : testDoc.getFields()) {
-      FieldInfo fieldInfo = fieldInfos.getOrAdd(field.name());
       IndexableFieldType ift = field.fieldType();
-      fieldInfo.setIndexOptions(ift.indexOptions());
-      if (ift.omitNorms()) {
-        fieldInfo.setOmitsNorms();
-      }
-      fieldInfo.setDocValuesType(ift.docValuesType());
+      fieldInfos.add(
+          new FieldInfo(
+              field.name(),
+              -1,
+              false,
+              ift.omitNorms(),
+              false,
+              ift.indexOptions(),
+              ift.docValuesType(),
+              -1,
+              new HashMap<>(),
+              0,
+              0,
+              0,
+              0,
+              VectorValues.SearchStrategy.NONE,
+              field.name().equals(softDeletesFieldName)));
     }
     dir = newDirectory();
     IndexWriterConfig conf =
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexOptions.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexOptions.java
index 1b11703..36c5f4b 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexOptions.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexOptions.java
@@ -20,7 +20,6 @@ import java.io.IOException;
 import java.util.Collections;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
-import org.apache.lucene.document.IntPoint;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.IOUtils;
@@ -31,51 +30,29 @@ public class TestIndexOptions extends LuceneTestCase {
   public void testChangeIndexOptionsViaAddDocument() throws IOException {
     for (IndexOptions from : IndexOptions.values()) {
       for (IndexOptions to : IndexOptions.values()) {
-        for (boolean preExisting : new boolean[] {false, true}) {
-          for (boolean onNewSegment : new boolean[] {false, true}) {
-            doTestChangeIndexOptionsViaAddDocument(preExisting, onNewSegment, from, to);
-          }
-        }
+        doTestChangeIndexOptionsViaAddDocument(from, to);
       }
     }
   }
 
-  private void doTestChangeIndexOptionsViaAddDocument(
-      boolean preExistingField, boolean onNewSegment, IndexOptions from, IndexOptions to)
+  private void doTestChangeIndexOptionsViaAddDocument(IndexOptions from, IndexOptions to)
       throws IOException {
     Directory dir = newDirectory();
     IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
-    if (preExistingField) {
-      w.addDocument(Collections.singleton(new IntPoint("foo", 1)));
-      if (onNewSegment) {
-        DirectoryReader.open(w).close();
-      }
-    }
     FieldType ft1 = new FieldType(TextField.TYPE_STORED);
     ft1.setIndexOptions(from);
     w.addDocument(Collections.singleton(new Field("foo", "bar", ft1)));
-    if (onNewSegment) {
-      DirectoryReader.open(w).close();
-    }
     FieldType ft2 = new FieldType(TextField.TYPE_STORED);
     ft2.setIndexOptions(to);
-    if (from == IndexOptions.NONE || to == IndexOptions.NONE || from == to) {
+    if (from == to) {
       w.addDocument(Collections.singleton(new Field("foo", "bar", ft2))); // no exception
-      w.forceMerge(1);
-      try (LeafReader r = getOnlyLeafReader(DirectoryReader.open(w))) {
-        IndexOptions expected = from == IndexOptions.NONE ? to : from;
-        assertEquals(expected, r.getFieldInfos().fieldInfo("foo").getIndexOptions());
-      }
     } else {
       IllegalArgumentException e =
           expectThrows(
               IllegalArgumentException.class,
               () -> w.addDocument(Collections.singleton(new Field("foo", "bar", ft2))));
       assertEquals(
-          "cannot change field \"foo\" from index options="
-              + from
-              + " to inconsistent index options="
-              + to,
+          "Inconsistency of field data structures across documents for field [foo] of doc [1].",
           e.getMessage());
     }
     w.close();
@@ -105,7 +82,7 @@ public class TestIndexOptions extends LuceneTestCase {
     w2.addDocument(Collections.singleton(new Field("foo", "bar", ft2)));
 
     try (CodecReader cr = (CodecReader) getOnlyLeafReader(DirectoryReader.open(w2))) {
-      if (from == IndexOptions.NONE || to == IndexOptions.NONE || from == to) {
+      if (from == to) {
         w1.addIndexes(cr); // no exception
         w1.forceMerge(1);
         try (LeafReader r = getOnlyLeafReader(DirectoryReader.open(w1))) {
@@ -150,7 +127,7 @@ public class TestIndexOptions extends LuceneTestCase {
     w2.addDocument(Collections.singleton(new Field("foo", "bar", ft2)));
     w2.close();
 
-    if (from == IndexOptions.NONE || to == IndexOptions.NONE || from == to) {
+    if (from == to) {
       w1.addIndexes(dir2); // no exception
       w1.forceMerge(1);
       try (LeafReader r = getOnlyLeafReader(DirectoryReader.open(w1))) {
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
index 0aba691..ae10636 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
@@ -2824,7 +2824,9 @@ public class TestIndexSorting extends LuceneTestCase {
         w.addDocument(doc);
         doc.add(dvs.get(j));
         exc = expectThrows(IllegalArgumentException.class, () -> w.addDocument(doc));
-        assertThat(exc.getMessage(), containsString("cannot change DocValues type"));
+        assertEquals(
+            "Inconsistency of field data structures across documents for field [field] of doc [2].",
+            exc.getMessage());
         w.rollback();
         IOUtils.close(w);
       }
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
index c48ce60..266b21d 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
@@ -16,6 +16,8 @@
  */
 package org.apache.lucene.index;
 
+import static org.apache.lucene.index.DocHelper.TEXT_TYPE_STORED_WITH_TVS;
+
 import com.carrotsearch.randomizedtesting.generators.RandomPicks;
 import java.io.ByteArrayOutputStream;
 import java.io.Closeable;
@@ -2008,7 +2010,7 @@ public class TestIndexWriter extends LuceneTestCase {
         NullPointerException.class,
         () -> {
           Document broke = new Document();
-          broke.add(newTextField("test", "broken", Field.Store.NO));
+          broke.add(new Field("test", "broken", TEXT_TYPE_STORED_WITH_TVS));
           iw.addDocument(broke);
         });
 
@@ -2820,16 +2822,16 @@ public class TestIndexWriter extends LuceneTestCase {
       IndexWriter w = new IndexWriter(dir, iwc);
       Document d = new Document();
       d.add(new StringField("id", "1", Field.Store.YES));
-      d.add(new NumericDocValuesField("id", 1));
+      d.add(new NumericDocValuesField("nvd", 1));
       w.addDocument(d);
       d = new Document();
       d.add(new StringField("id", "2", Field.Store.YES));
-      d.add(new NumericDocValuesField("id", 2));
+      d.add(new NumericDocValuesField("nvd", 2));
       w.addDocument(d);
       w.flush();
       d = new Document();
       d.add(new StringField("id", "1", Field.Store.YES));
-      d.add(new NumericDocValuesField("id", 1));
+      d.add(new NumericDocValuesField("nvd", 1));
       w.updateDocument(new Term("id", "1"), d);
       w.commit();
       Set<String> files = new HashSet<>(Arrays.asList(dir.listAll()));
@@ -2838,12 +2840,12 @@ public class TestIndexWriter extends LuceneTestCase {
         if (random().nextBoolean()) {
           d = new Document();
           d.add(new StringField("id", "1", Field.Store.YES));
-          d.add(new NumericDocValuesField("id", 1));
+          d.add(new NumericDocValuesField("nvd", 1));
           w.updateDocument(new Term("id", "1"), d);
         } else if (random().nextBoolean()) {
           w.deleteDocuments(new Term("id", "2"));
         } else {
-          w.updateNumericDocValue(new Term("id", "1"), "id", 2);
+          w.updateNumericDocValue(new Term("id", "1"), "nvd", 2);
         }
         w.prepareCommit();
         List<String> newFiles = new ArrayList<>(Arrays.asList(dir.listAll()));
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDeleteByQuery.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDeleteByQuery.java
deleted file mode 100644
index d65d812..0000000
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDeleteByQuery.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.index;
-
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.search.MatchAllDocsQuery;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.LuceneTestCase;
-
-public class TestIndexWriterDeleteByQuery extends LuceneTestCase {
-
-  // LUCENE-6379
-  public void testDeleteMatchAllDocsQuery() throws Exception {
-    Directory dir = newMaybeVirusCheckingDirectory();
-    IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
-    Document doc = new Document();
-    // Norms are disabled:
-    doc.add(newStringField("field", "foo", Field.Store.NO));
-    w.addDocument(doc);
-    DirectoryReader r = DirectoryReader.open(w);
-    FieldInfo fi = FieldInfos.getMergedFieldInfos(r).fieldInfo("field");
-    assertNotNull(fi);
-    assertFalse(fi.hasNorms());
-    assertEquals(1, r.numDocs());
-    assertEquals(1, r.maxDoc());
-
-    w.deleteDocuments(new MatchAllDocsQuery());
-    DirectoryReader r2 = DirectoryReader.openIfChanged(r);
-    r.close();
-
-    assertNotNull(r2);
-    assertEquals(0, r2.numDocs());
-    assertEquals(0, r2.maxDoc());
-
-    // Confirm the omitNorms bit is in fact no longer set:
-    doc = new Document();
-    // Norms are disabled:
-    doc.add(newTextField("field", "foo", Field.Store.NO));
-    w.addDocument(doc);
-
-    DirectoryReader r3 = DirectoryReader.openIfChanged(r2);
-    r2.close();
-    assertNotNull(r3);
-    assertEquals(1, r3.numDocs());
-    assertEquals(1, r3.maxDoc());
-
-    // Make sure norms can come back to life for a field after deleting by MatchAllDocsQuery:
-    fi = FieldInfos.getMergedFieldInfos(r3).fieldInfo("field");
-    assertNotNull(fi);
-    assertTrue(fi.hasNorms());
-    r3.close();
-    w.close();
-    dir.close();
-  }
-}
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java
index 2b77a0b..dd97c44 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java
@@ -156,7 +156,7 @@ public class TestIndexWriterReader extends LuceneTestCase {
 
     Document newDoc = r1.document(10);
     newDoc.removeField("id");
-    newDoc.add(newStringField("id", Integer.toString(8000), Field.Store.YES));
+    newDoc.add(new Field("id", Integer.toString(8000), DocHelper.STRING_TYPE_STORED_WITH_TVS));
     writer.updateDocument(new Term("id", id10), newDoc);
     assertFalse(r1.isCurrent());
 
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestMixedDocValuesUpdates.java b/lucene/core/src/test/org/apache/lucene/index/TestMixedDocValuesUpdates.java
index eaa6816..d5e0826 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestMixedDocValuesUpdates.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestMixedDocValuesUpdates.java
@@ -456,7 +456,7 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
     for (int i = 0; i < numDocs; i++) {
       Document doc = new Document();
       doc.add(new StringField("id", "" + i, Store.YES));
-      doc.add(new NumericDocValuesField("id", i));
+      doc.add(new NumericDocValuesField("numericId", i));
       doc.add(new BinaryDocValuesField("binaryId", new BytesRef(new byte[] {(byte) i})));
       writer.addDocument(doc);
       if (random().nextBoolean()) {
@@ -467,19 +467,19 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
     doUpdate(
         new Term("id", "" + doc),
         writer,
-        new NumericDocValuesField("id", doc + 1),
+        new NumericDocValuesField("numericId", doc + 1),
         new BinaryDocValuesField("binaryId", new BytesRef(new byte[] {(byte) (doc + 1)})));
     IndexReader reader = writer.getReader();
-    NumericDocValues idValues = null;
+    NumericDocValues numericIdValues = null;
     BinaryDocValues binaryIdValues = null;
     for (LeafReaderContext c : reader.leaves()) {
       TopDocs topDocs =
           new IndexSearcher(c.reader()).search(new TermQuery(new Term("id", "" + doc)), 10);
       if (topDocs.totalHits.value == 1) {
-        assertNull(idValues);
+        assertNull(numericIdValues);
         assertNull(binaryIdValues);
-        idValues = c.reader().getNumericDocValues("id");
-        assertEquals(topDocs.scoreDocs[0].doc, idValues.advance(topDocs.scoreDocs[0].doc));
+        numericIdValues = c.reader().getNumericDocValues("numericId");
+        assertEquals(topDocs.scoreDocs[0].doc, numericIdValues.advance(topDocs.scoreDocs[0].doc));
         binaryIdValues = c.reader().getBinaryDocValues("binaryId");
         assertEquals(topDocs.scoreDocs[0].doc, binaryIdValues.advance(topDocs.scoreDocs[0].doc));
       } else {
@@ -487,10 +487,10 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
       }
     }
 
-    assertNotNull(idValues);
+    assertNotNull(numericIdValues);
     assertNotNull(binaryIdValues);
 
-    assertEquals(doc + 1, idValues.longValue());
+    assertEquals(doc + 1, numericIdValues.longValue());
     assertEquals(new BytesRef(new byte[] {(byte) (doc + 1)}), binaryIdValues.binaryValue());
     IOUtils.close(reader, writer, dir);
   }
@@ -709,7 +709,7 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
       IllegalArgumentException iae =
           expectThrows(IllegalArgumentException.class, () -> writer.addDocument(doc1));
       assertEquals(
-          "cannot change DocValues type from NUMERIC to BINARY for field \"not_existing\"",
+          "cannot change field \"not_existing\" from doc values type=NUMERIC to inconsistent doc values type=BINARY",
           iae.getMessage());
 
       iae =
@@ -720,12 +720,12 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
                       new Term("id", "1"),
                       new BinaryDocValuesField("not_existing", new BytesRef())));
       assertEquals(
-          "cannot change DocValues type from NUMERIC to BINARY for field \"not_existing\"",
+          "Can't update [BINARY] doc values; the field [not_existing] has inconsistent doc values' type of [NUMERIC].",
           iae.getMessage());
     }
   }
 
-  public void testUpdateFieldWithNoPreviousDocValues() throws IOException {
+  public void testUpdateFieldWithNoPreviousDocValuesThrowsError() throws IOException {
     IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
     try (Directory dir = newDirectory();
         IndexWriter writer = new IndexWriter(dir, conf)) {
@@ -740,13 +740,14 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
       } else if (random().nextBoolean()) {
         writer.commit();
       }
-      writer.updateDocValues(new Term("id", "1"), new NumericDocValuesField("id", 1));
-      try (DirectoryReader reader = writer.getReader()) {
-        NumericDocValues id = reader.leaves().get(0).reader().getNumericDocValues("id");
-        assertNotNull(id);
-        assertTrue(id.advanceExact(0));
-        assertEquals(1, id.longValue());
-      }
+      IllegalArgumentException exception =
+          expectThrows(
+              IllegalArgumentException.class,
+              () ->
+                  writer.updateDocValues(new Term("id", "1"), new NumericDocValuesField("id", 1)));
+      assertEquals(
+          "Can't update [NUMERIC] doc values; the field [id] has inconsistent doc values' type of [NONE].",
+          exception.getMessage());
     }
   }
 }
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java b/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java
index ed6d9aa..fd431a1 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java
@@ -1211,23 +1211,34 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
     conf.setMergePolicy(NoMergePolicy.INSTANCE);
     IndexWriter writer = new IndexWriter(dir, conf);
 
-    // first segment with NDV
+    // first segment with ndv and ndv2 fields
     Document doc = new Document();
     doc.add(new StringField("id", "doc0", Store.NO));
-    doc.add(new StringField("ndv", "mock-value", Store.NO));
     doc.add(new NumericDocValuesField("ndv", 5));
+    doc.add(new StringField("ndv2", "10", Store.NO));
+    doc.add(new NumericDocValuesField("ndv2", 10));
     writer.addDocument(doc);
     writer.commit();
 
-    // second segment with no NDV
+    // second segment with no ndv and ndv2 fields
     doc = new Document();
     doc.add(new StringField("id", "doc1", Store.NO));
-    doc.add(new StringField("ndv", "mock-value", Store.NO));
     writer.addDocument(doc);
     writer.commit();
 
-    // update document in the second segment
+    // update docValues of "ndv" field in the second segment
+    // since global "ndv" field is docValues only field this is allowed
     writer.updateNumericDocValue(new Term("id", "doc1"), "ndv", 5L);
+
+    // update docValues of "ndv2" field in the second segment
+    // since global "ndv2" field is not docValues only field this NOT allowed
+    IllegalArgumentException exception =
+        expectThrows(
+            IllegalArgumentException.class,
+            () -> writer.updateNumericDocValue(new Term("id", "doc1"), "ndv2", 10L));
+    String expectedErrMsg =
+        "Can't update [NUMERIC] doc values; the field [ndv2] must be doc values only field, but is also indexed with postings.";
+    assertEquals(expectedErrMsg, exception.getMessage());
     writer.close();
 
     DirectoryReader reader = DirectoryReader.open(dir);
@@ -1256,13 +1267,21 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
     doc.add(new NumericDocValuesField("f", 5));
     writer.addDocument(doc);
     writer.commit();
-    writer.updateNumericDocValue(new Term("f", "mock-value"), "f", 17L);
+
+    IllegalArgumentException exception =
+        expectThrows(
+            IllegalArgumentException.class,
+            () -> writer.updateNumericDocValue(new Term("f", "mock-value"), "f", 17L));
+    String expectedErrMsg =
+        "Can't update [NUMERIC] doc values; the field [f] must be doc values only field, but is also indexed with postings.";
+    assertEquals(expectedErrMsg, exception.getMessage());
+
     writer.close();
 
     DirectoryReader r = DirectoryReader.open(dir);
     NumericDocValues ndv = r.leaves().get(0).reader().getNumericDocValues("f");
     assertEquals(0, ndv.nextDoc());
-    assertEquals(17, ndv.longValue());
+    assertEquals(5, ndv.longValue());
     r.close();
 
     dir.close();
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestOmitNorms.java b/lucene/core/src/test/org/apache/lucene/index/TestOmitNorms.java
index 9d36651..f0b5583 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestOmitNorms.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestOmitNorms.java
@@ -16,9 +16,6 @@
  */
 package org.apache.lucene.index;
 
-import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
-
-import java.io.IOException;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
@@ -27,58 +24,11 @@ import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.TestUtil;
 
 public class TestOmitNorms extends LuceneTestCase {
-  // Tests whether the DocumentWriter correctly enable the
-  // omitNorms bit in the FieldInfo
-  public void testOmitNorms() throws Exception {
-    Directory ram = newDirectory();
-    Analyzer analyzer = new MockAnalyzer(random());
-    IndexWriter writer = new IndexWriter(ram, newIndexWriterConfig(analyzer));
-    Document d = new Document();
-
-    // this field will have norms
-    Field f1 = newTextField("f1", "This field has norms", Field.Store.NO);
-    d.add(f1);
-
-    // this field will NOT have norms
-    FieldType customType = new FieldType(TextField.TYPE_NOT_STORED);
-    customType.setOmitNorms(true);
-    Field f2 = newField("f2", "This field has NO norms in all docs", customType);
-    d.add(f2);
-
-    writer.addDocument(d);
-    writer.forceMerge(1);
-    // now we add another document which has term freq for field f2 and not for f1 and verify if the
-    // SegmentMerger
-    // keep things constant
-    d = new Document();
-
-    // Reverse
-    d.add(newField("f1", "This field has norms", customType));
-
-    d.add(newTextField("f2", "This field has NO norms in all docs", Field.Store.NO));
-
-    writer.addDocument(d);
-
-    // force merge
-    writer.forceMerge(1);
-    // flush
-    writer.close();
-
-    LeafReader reader = getOnlyLeafReader(DirectoryReader.open(ram));
-    FieldInfos fi = reader.getFieldInfos();
-    assertTrue("OmitNorms field bit should be set.", fi.fieldInfo("f1").omitsNorms());
-    assertTrue("OmitNorms field bit should be set.", fi.fieldInfo("f2").omitsNorms());
-
-    reader.close();
-    ram.close();
-  }
 
-  // Tests whether merging of docs that have different
-  // omitNorms for the same field works
-  public void testMixedMerge() throws Exception {
+  // Tests that merging of docs with different omitNorms throws error
+  public void testMixedMergeThrowsError() throws Exception {
     Directory ram = newDirectory();
     Analyzer analyzer = new MockAnalyzer(random());
     IndexWriter writer =
@@ -90,42 +40,42 @@ public class TestOmitNorms extends LuceneTestCase {
     Document d = new Document();
 
     // this field will have norms
-    Field f1 = newTextField("f1", "This field has norms", Field.Store.NO);
+    FieldType fieldType1 = new FieldType(TextField.TYPE_NOT_STORED);
+    fieldType1.setOmitNorms(false);
+    fieldType1.setStoreTermVectors(false);
+    Field f1 = new Field("f1", "This field has norms", fieldType1);
     d.add(f1);
 
     // this field will NOT have norms
-    FieldType customType = new FieldType(TextField.TYPE_NOT_STORED);
-    customType.setOmitNorms(true);
-    Field f2 = newField("f2", "This field has NO norms in all docs", customType);
+    FieldType fieldType2 = new FieldType(TextField.TYPE_NOT_STORED);
+    fieldType2.setOmitNorms(true);
+    fieldType2.setStoreTermVectors(false);
+    Field f2 = new Field("f2", "This field has NO norms in all docs", fieldType2);
     d.add(f2);
 
     for (int i = 0; i < 30; i++) {
       writer.addDocument(d);
     }
 
-    // now we add another document which has norms for field f2 and not for f1 and verify if the
-    // SegmentMerger
-    // keep things constant
-    d = new Document();
+    // reverse omitNorms options for f1 and f2
+    Document d2 = new Document();
+    d2.add(new Field("f1", "This field has NO norms", fieldType2));
+    d2.add(new Field("f2", "This field has norms", fieldType1));
 
-    // Reverese
-    d.add(newField("f1", "This field has norms", customType));
+    IllegalArgumentException exception =
+        expectThrows(IllegalArgumentException.class, () -> writer.addDocument(d2));
+    assertEquals(
+        "cannot change field \"f1\" from omitNorms=false to inconsistent omitNorms=true",
+        exception.getMessage());
 
-    d.add(newTextField("f2", "This field has NO norms in all docs", Field.Store.NO));
-
-    for (int i = 0; i < 30; i++) {
-      writer.addDocument(d);
-    }
-
-    // force merge
     writer.forceMerge(1);
-    // flush
     writer.close();
 
     LeafReader reader = getOnlyLeafReader(DirectoryReader.open(ram));
     FieldInfos fi = reader.getFieldInfos();
-    assertTrue("OmitNorms field bit should be set.", fi.fieldInfo("f1").omitsNorms());
-    assertTrue("OmitNorms field bit should be set.", fi.fieldInfo("f2").omitsNorms());
+    // assert original omitNorms
+    assertTrue("OmitNorms field bit must not be set.", fi.fieldInfo("f1").omitsNorms() == false);
+    assertTrue("OmitNorms field bit must be set.", fi.fieldInfo("f2").omitsNorms());
 
     reader.close();
     ram.close();
@@ -223,110 +173,4 @@ public class TestOmitNorms extends LuceneTestCase {
     assertNoNrm(ram);
     ram.close();
   }
-
-  /**
-   * Tests various combinations of omitNorms=true/false, the field not existing at all, ensuring
-   * that only omitNorms is 'viral'. Internally checks that MultiNorms.norms() is consistent
-   * (returns the same bytes) as the fully merged equivalent.
-   */
-  public void testOmitNormsCombos() throws IOException {
-    // indexed with norms
-    FieldType customType = new FieldType(TextField.TYPE_STORED);
-    Field norms = new Field("foo", "a", customType);
-    // indexed without norms
-    FieldType customType1 = new FieldType(TextField.TYPE_STORED);
-    customType1.setOmitNorms(true);
-    Field noNorms = new Field("foo", "a", customType1);
-    // not indexed, but stored
-    FieldType customType2 = new FieldType();
-    customType2.setStored(true);
-    Field noIndex = new Field("foo", "a", customType2);
-    // not indexed but stored, omitNorms is set
-    FieldType customType3 = new FieldType();
-    customType3.setStored(true);
-    customType3.setOmitNorms(true);
-    Field noNormsNoIndex = new Field("foo", "a", customType3);
-    // not indexed nor stored (doesnt exist at all, we index a different field instead)
-    Field emptyNorms = new Field("bar", "a", customType);
-
-    assertNotNull(getNorms("foo", norms, norms));
-    assertNull(getNorms("foo", norms, noNorms));
-    assertNotNull(getNorms("foo", norms, noIndex));
-    assertNotNull(getNorms("foo", norms, noNormsNoIndex));
-    assertNotNull(getNorms("foo", norms, emptyNorms));
-    assertNull(getNorms("foo", noNorms, noNorms));
-    assertNull(getNorms("foo", noNorms, noIndex));
-    assertNull(getNorms("foo", noNorms, noNormsNoIndex));
-    assertNull(getNorms("foo", noNorms, emptyNorms));
-    assertNull(getNorms("foo", noIndex, noIndex));
-    assertNull(getNorms("foo", noIndex, noNormsNoIndex));
-    assertNull(getNorms("foo", noIndex, emptyNorms));
-    assertNull(getNorms("foo", noNormsNoIndex, noNormsNoIndex));
-    assertNull(getNorms("foo", noNormsNoIndex, emptyNorms));
-    assertNull(getNorms("foo", emptyNorms, emptyNorms));
-  }
-
-  /**
-   * Indexes at least 1 document with f1, and at least 1 document with f2. returns the norms for
-   * "field".
-   */
-  NumericDocValues getNorms(String field, Field f1, Field f2) throws IOException {
-    Directory dir = newDirectory();
-    IndexWriterConfig iwc =
-        newIndexWriterConfig(new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy());
-    RandomIndexWriter riw = new RandomIndexWriter(random(), dir, iwc);
-
-    // add f1
-    Document d = new Document();
-    d.add(f1);
-    riw.addDocument(d);
-
-    // add f2
-    d = new Document();
-    d.add(f2);
-    riw.addDocument(d);
-
-    // add a mix of f1's and f2's
-    int numExtraDocs = TestUtil.nextInt(random(), 1, 1000);
-    for (int i = 0; i < numExtraDocs; i++) {
-      d = new Document();
-      d.add(random().nextBoolean() ? f1 : f2);
-      riw.addDocument(d);
-    }
-
-    IndexReader ir1 = riw.getReader();
-    // todo: generalize
-    NumericDocValues norms1 = MultiDocValues.getNormValues(ir1, field);
-
-    // fully merge and validate MultiNorms against single segment.
-    riw.forceMerge(1);
-    DirectoryReader ir2 = riw.getReader();
-    NumericDocValues norms2 = getOnlyLeafReader(ir2).getNormValues(field);
-
-    if (norms1 == null) {
-      assertNull(norms2);
-    } else {
-      while (true) {
-        int norms1DocID = norms1.nextDoc();
-        int norms2DocID = norms2.nextDoc();
-        while (norms1DocID < norms2DocID) {
-          assertEquals(0, norms1.longValue());
-          norms1DocID = norms1.nextDoc();
-        }
-        while (norms2DocID < norms1DocID) {
-          assertEquals(0, norms2.longValue());
-          norms2DocID = norms2.nextDoc();
-        }
-        if (norms1.docID() == NO_MORE_DOCS) {
-          break;
-        }
-        assertEquals(norms1.longValue(), norms2.longValue());
-      }
-    }
-    ir1.close();
-    ir2.close();
-    riw.close();
-    dir.close();
-    return norms1;
-  }
 }
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java b/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
index bac9dc9..83c6ce4 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
@@ -74,7 +74,8 @@ public class TestPointValues extends LuceneTestCase {
               w.addDocument(doc);
             });
     assertEquals(
-        "cannot change point dimension count from 1 to 2 for field=\"dim\"", expected.getMessage());
+        "Inconsistency of field data structures across documents for field [dim] of doc [0].",
+        expected.getMessage());
     w.close();
     dir.close();
   }
@@ -96,8 +97,8 @@ public class TestPointValues extends LuceneTestCase {
               w.addDocument(doc2);
             });
     assertEquals(
-        "cannot change point dimension count from 1 to 2 for field=\"dim\"", expected.getMessage());
-
+        "Inconsistency of field data structures across documents for field [dim] of doc [1].",
+        expected.getMessage());
     w.close();
     dir.close();
   }
@@ -120,8 +121,9 @@ public class TestPointValues extends LuceneTestCase {
               w.addDocument(doc2);
             });
     assertEquals(
-        "cannot change point dimension count from 1 to 2 for field=\"dim\"", expected.getMessage());
-
+        "cannot change field \"dim\" from points dimensionCount=1, indexDimensionCount=1, numBytes=4 "
+            + "to inconsistent dimensionCount=2, indexDimensionCount=2, numBytes=4",
+        expected.getMessage());
     w.close();
     dir.close();
   }
@@ -146,8 +148,9 @@ public class TestPointValues extends LuceneTestCase {
               w2.addDocument(doc2);
             });
     assertEquals(
-        "cannot change point dimension count from 1 to 2 for field=\"dim\"", expected.getMessage());
-
+        "cannot change field \"dim\" from points dimensionCount=1, indexDimensionCount=1, numBytes=4 "
+            + "to inconsistent dimensionCount=2, indexDimensionCount=2, numBytes=4",
+        expected.getMessage());
     w2.close();
     dir.close();
   }
@@ -172,9 +175,11 @@ public class TestPointValues extends LuceneTestCase {
             () -> {
               w2.addIndexes(new Directory[] {dir});
             });
-    assertEquals(
-        "cannot change point dimension count from 2 to 1 for field=\"dim\"", expected.getMessage());
 
+    assertEquals(
+        "cannot change field \"dim\" from points dimensionCount=2, indexDimensionCount=2, numBytes=4 "
+            + "to inconsistent dimensionCount=1, indexDimensionCount=1, numBytes=4",
+        expected.getMessage());
     IOUtils.close(w2, dir, dir2);
   }
 
@@ -200,8 +205,9 @@ public class TestPointValues extends LuceneTestCase {
               w2.addIndexes(new CodecReader[] {(CodecReader) getOnlyLeafReader(r)});
             });
     assertEquals(
-        "cannot change point dimension count from 2 to 1 for field=\"dim\"", expected.getMessage());
-
+        "cannot change field \"dim\" from points dimensionCount=2, indexDimensionCount=2, numBytes=4 "
+            + "to inconsistent dimensionCount=1, indexDimensionCount=1, numBytes=4",
+        expected.getMessage());
     IOUtils.close(r, w2, dir, dir2);
   }
 
@@ -228,8 +234,9 @@ public class TestPointValues extends LuceneTestCase {
               TestUtil.addIndexesSlowly(w2, r);
             });
     assertEquals(
-        "cannot change point dimension count from 2 to 1 for field=\"dim\"", expected.getMessage());
-
+        "cannot change field \"dim\" from points dimensionCount=2, indexDimensionCount=2, numBytes=4 "
+            + "to inconsistent dimensionCount=1, indexDimensionCount=1, numBytes=4",
+        expected.getMessage());
     IOUtils.close(r, w2, dir, dir2);
   }
 
@@ -247,8 +254,8 @@ public class TestPointValues extends LuceneTestCase {
               w.addDocument(doc);
             });
     assertEquals(
-        "cannot change point numBytes from 4 to 6 for field=\"dim\"", expected.getMessage());
-
+        "Inconsistency of field data structures across documents for field [dim] of doc [0].",
+        expected.getMessage());
     w.close();
     dir.close();
   }
@@ -270,8 +277,8 @@ public class TestPointValues extends LuceneTestCase {
               w.addDocument(doc2);
             });
     assertEquals(
-        "cannot change point numBytes from 4 to 6 for field=\"dim\"", expected.getMessage());
-
+        "Inconsistency of field data structures across documents for field [dim] of doc [1].",
+        expected.getMessage());
     w.close();
     dir.close();
   }
@@ -294,8 +301,9 @@ public class TestPointValues extends LuceneTestCase {
               w.addDocument(doc2);
             });
     assertEquals(
-        "cannot change point numBytes from 4 to 6 for field=\"dim\"", expected.getMessage());
-
+        "cannot change field \"dim\" from points dimensionCount=1, indexDimensionCount=1, numBytes=4 "
+            + "to inconsistent dimensionCount=1, indexDimensionCount=1, numBytes=6",
+        expected.getMessage());
     w.close();
     dir.close();
   }
@@ -321,8 +329,9 @@ public class TestPointValues extends LuceneTestCase {
               w2.addDocument(doc2);
             });
     assertEquals(
-        "cannot change point numBytes from 4 to 6 for field=\"dim\"", expected.getMessage());
-
+        "cannot change field \"dim\" from points dimensionCount=1, indexDimensionCount=1, numBytes=4 "
+            + "to inconsistent dimensionCount=1, indexDimensionCount=1, numBytes=6",
+        expected.getMessage());
     w2.close();
     dir.close();
   }
@@ -349,8 +358,9 @@ public class TestPointValues extends LuceneTestCase {
               w2.addIndexes(new Directory[] {dir});
             });
     assertEquals(
-        "cannot change point numBytes from 6 to 4 for field=\"dim\"", expected.getMessage());
-
+        "cannot change field \"dim\" from points dimensionCount=1, indexDimensionCount=1, numBytes=6 "
+            + "to inconsistent dimensionCount=1, indexDimensionCount=1, numBytes=4",
+        expected.getMessage());
     IOUtils.close(w2, dir, dir2);
   }
 
@@ -377,8 +387,9 @@ public class TestPointValues extends LuceneTestCase {
               w2.addIndexes(new CodecReader[] {(CodecReader) getOnlyLeafReader(r)});
             });
     assertEquals(
-        "cannot change point numBytes from 6 to 4 for field=\"dim\"", expected.getMessage());
-
+        "cannot change field \"dim\" from points dimensionCount=1, indexDimensionCount=1, numBytes=6 "
+            + "to inconsistent dimensionCount=1, indexDimensionCount=1, numBytes=4",
+        expected.getMessage());
     IOUtils.close(r, w2, dir, dir2);
   }
 
@@ -405,8 +416,9 @@ public class TestPointValues extends LuceneTestCase {
               TestUtil.addIndexesSlowly(w2, r);
             });
     assertEquals(
-        "cannot change point numBytes from 6 to 4 for field=\"dim\"", expected.getMessage());
-
+        "cannot change field \"dim\" from points dimensionCount=1, indexDimensionCount=1, numBytes=6 "
+            + "to inconsistent dimensionCount=1, indexDimensionCount=1, numBytes=4",
+        expected.getMessage());
     IOUtils.close(r, w2, dir, dir2);
   }
 
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java b/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java
index 52e5116..2187e42 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java
@@ -363,41 +363,6 @@ public class TestPostingsOffsets extends LuceneTestCase {
     dir.close();
   }
 
-  public void testWithUnindexedFields() throws Exception {
-    Directory dir = newDirectory();
-    RandomIndexWriter riw = new RandomIndexWriter(random(), dir, iwc);
-    for (int i = 0; i < 100; i++) {
-      Document doc = new Document();
-      // ensure at least one doc is indexed with offsets
-      if (i < 99 && random().nextInt(2) == 0) {
-        // stored only
-        FieldType ft = new FieldType();
-        ft.setStored(true);
-        doc.add(new Field("foo", "boo!", ft));
-      } else {
-        FieldType ft = new FieldType(TextField.TYPE_STORED);
-        ft.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
-        if (random().nextBoolean()) {
-          // store some term vectors for the checkindex cross-check
-          ft.setStoreTermVectors(true);
-          ft.setStoreTermVectorPositions(true);
-          ft.setStoreTermVectorOffsets(true);
-        }
-        doc.add(new Field("foo", "bar", ft));
-      }
-      riw.addDocument(doc);
-    }
-    CompositeReader ir = riw.getReader();
-    FieldInfos fis = FieldInfos.getMergedFieldInfos(ir);
-    assertEquals(
-        IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS,
-        fis.fieldInfo("foo").getIndexOptions());
-    ir.close();
-    ir.close();
-    riw.close();
-    dir.close();
-  }
-
   public void testAddFieldTwice() throws Exception {
     Directory dir = newDirectory();
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSortingCodecReader.java b/lucene/core/src/test/org/apache/lucene/index/TestSortingCodecReader.java
index 2bc6b16..939c3cb 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestSortingCodecReader.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestSortingCodecReader.java
@@ -114,8 +114,8 @@ public class TestSortingCodecReader extends LuceneTestCase {
         for (int i = 0; i < numDocs; i++) {
           int docId = docIds.get(i);
           Document doc = new Document();
-          doc.add(new StringField("id", Integer.toString(docId), Field.Store.YES));
-          doc.add(new LongPoint("id", docId));
+          doc.add(new StringField("string_id", Integer.toString(docId), Field.Store.YES));
+          doc.add(new LongPoint("point_id", docId));
           String s = RandomStrings.randomRealisticUnicodeOfLength(random(), 25);
           doc.add(new TextField("text_field", s, Field.Store.YES));
           doc.add(new BinaryDocValuesField("text_field", new BytesRef(s)));
@@ -150,7 +150,7 @@ public class TestSortingCodecReader extends LuceneTestCase {
           iw.addDocument(doc);
           if (i > 0 && random().nextInt(5) == 0) {
             final int id = RandomPicks.randomFrom(random(), docIds.subList(0, i));
-            iw.deleteDocuments(new Term("id", Integer.toString(id)));
+            iw.deleteDocuments(new Term("string_id", Integer.toString(id)));
           }
         }
         iw.commit();
@@ -242,13 +242,15 @@ public class TestSortingCodecReader extends LuceneTestCase {
                       .terms("term_vectors")
                       .iterator()
                       .seekExact(new BytesRef("test" + ids.longValue())));
-              assertEquals(Long.toString(ids.longValue()), leaf.document(idNext).get("id"));
+              assertEquals(Long.toString(ids.longValue()), leaf.document(idNext).get("string_id"));
               IndexSearcher searcher = new IndexSearcher(r);
-              TopDocs result = searcher.search(LongPoint.newExactQuery("id", ids.longValue()), 1);
+              TopDocs result =
+                  searcher.search(LongPoint.newExactQuery("point_id", ids.longValue()), 1);
               assertEquals(1, result.totalHits.value);
               assertEquals(idNext, result.scoreDocs[0].doc);
 
-              result = searcher.search(new TermQuery(new Term("id", "" + ids.longValue())), 1);
+              result =
+                  searcher.search(new TermQuery(new Term("string_id", "" + ids.longValue())), 1);
               assertEquals(1, result.totalHits.value);
               assertEquals(idNext, result.scoreDocs[0].doc);
             }
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java b/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java
index 1fef712..2fcb01c 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java
@@ -26,6 +26,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
+import java.util.concurrent.ConcurrentHashMap;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -39,6 +40,7 @@ import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
+import org.junit.Before;
 
 public class TestStressIndexing2 extends LuceneTestCase {
   static int maxFields = 4;
@@ -47,6 +49,14 @@ public class TestStressIndexing2 extends LuceneTestCase {
   static int mergeFactor = 3;
   static int maxBufferedDocs = 3;
   static int seed = 0;
+  private static Map<String, FieldType> fieldTypes;
+
+  @Before
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    fieldTypes = new ConcurrentHashMap<>();
+  }
 
   public void testRandomIWReader() throws Throwable {
     Directory dir = newMaybeVirusCheckingDirectory();
@@ -128,7 +138,6 @@ public class TestStressIndexing2 extends LuceneTestCase {
     }
   }
 
-  static Term idTerm = new Term("id", "");
   IndexingThread[] threads;
   static Comparator<IndexableField> fieldNameComparator =
       new Comparator<IndexableField>() {
@@ -807,71 +816,58 @@ public class TestStressIndexing2 extends LuceneTestCase {
       Field idField = newField("id", idString, customType1);
       fields.add(idField);
 
-      Map<String, FieldType> tvTypes = new HashMap<>();
-
       int nFields = nextInt(maxFields);
       for (int i = 0; i < nFields; i++) {
-
         String fieldName = "f" + nextInt(100);
-        FieldType customType;
-
-        // Use the same term vector settings if we already
-        // added this field to the doc:
-        FieldType oldTVType = tvTypes.get(fieldName);
-        if (oldTVType != null) {
-          customType = new FieldType(oldTVType);
-        } else {
-          customType = new FieldType();
-          switch (nextInt(4)) {
-            case 0:
-              break;
-            case 1:
-              customType.setStoreTermVectors(true);
-              break;
-            case 2:
-              customType.setStoreTermVectors(true);
-              customType.setStoreTermVectorPositions(true);
-              break;
-            case 3:
-              customType.setStoreTermVectors(true);
-              customType.setStoreTermVectorOffsets(true);
-              break;
-          }
-          FieldType newType = new FieldType(customType);
-          newType.freeze();
-          tvTypes.put(fieldName, newType);
-        }
-
-        switch (nextInt(4)) {
-          case 0:
-            customType.setStored(true);
-            customType.setOmitNorms(true);
-            customType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
-            customType.freeze();
-            fields.add(newField(fieldName, getString(1), customType));
-            break;
-          case 1:
-            customType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
-            customType.setTokenized(true);
-            customType.freeze();
-            fields.add(newField(fieldName, getString(0), customType));
-            break;
-          case 2:
-            customType.setStored(true);
-            customType.setStoreTermVectors(false);
-            customType.setStoreTermVectorOffsets(false);
-            customType.setStoreTermVectorPositions(false);
-            customType.freeze();
-            fields.add(newField(fieldName, getString(0), customType));
-            break;
-          case 3:
-            customType.setStored(true);
-            customType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
-            customType.setTokenized(true);
-            customType.freeze();
-            fields.add(newField(fieldName, getString(bigFieldSize), customType));
-            break;
-        }
+        // Use the same field type if we already added this field to the index
+        FieldType fieldType =
+            fieldTypes.computeIfAbsent(
+                fieldName,
+                fn -> {
+                  FieldType ft = new FieldType();
+                  switch (nextInt(4)) {
+                    case 0:
+                      break;
+                    case 1:
+                      ft.setStoreTermVectors(true);
+                      break;
+                    case 2:
+                      ft.setStoreTermVectors(true);
+                      ft.setStoreTermVectorPositions(true);
+                      break;
+                    case 3:
+                      ft.setStoreTermVectors(true);
+                      ft.setStoreTermVectorOffsets(true);
+                      break;
+                  }
+                  switch (nextInt(4)) {
+                    case 0:
+                      ft.setStored(true);
+                      ft.setOmitNorms(true);
+                      ft.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
+                      break;
+                    case 1:
+                      ft.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
+                      ft.setTokenized(true);
+                      break;
+                    case 2:
+                      ft.setStored(true);
+                      ft.setStoreTermVectors(false);
+                      ft.setStoreTermVectorOffsets(false);
+                      ft.setStoreTermVectorPositions(false);
+                      break;
+                    case 3:
+                      ft.setStored(true);
+                      ft.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
+                      ft.setTokenized(true);
+                      break;
+                  }
+                  ft.freeze();
+                  return ft;
+                });
+        int nTokens = nextInt(3);
+        nTokens = nTokens < 2 ? nTokens : bigFieldSize;
+        fields.add(newField(fieldName, getString(nTokens), fieldType));
       }
 
       if (sameFieldOrder) {
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsWriter.java b/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsWriter.java
index 7fa98d4..c019db8 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsWriter.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsWriter.java
@@ -16,6 +16,9 @@
  */
 package org.apache.lucene.index;
 
+import static org.hamcrest.CoreMatchers.anyOf;
+import static org.hamcrest.CoreMatchers.startsWith;
+
 import java.io.IOException;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.CachingTokenFilter;
@@ -35,6 +38,7 @@ import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
+import org.hamcrest.MatcherAssert;
 
 /** tests for writing term vectors */
 public class TestTermVectorsWriter extends LuceneTestCase {
@@ -649,18 +653,15 @@ public class TestTermVectorsWriter extends LuceneTestCase {
 
     // ensure broken doc hits exception
     IllegalArgumentException expected =
-        expectThrows(
-            IllegalArgumentException.class,
-            () -> {
-              iw.addDocument(doc);
-            });
+        expectThrows(IllegalArgumentException.class, () -> iw.addDocument(doc));
     assertNotNull(expected.getMessage());
-    assertTrue(
-        expected
-            .getMessage()
-            .startsWith(
-                "all instances of a given field name must have the same term vectors settings"));
-
+    MatcherAssert.assertThat(
+        expected.getMessage(),
+        anyOf(
+            startsWith(
+                "all instances of a given field name must have the same term vectors settings"),
+            startsWith(
+                "Inconsistency of field data structures across documents for field [field]")));
     // ensure good docs are still ok
     IndexReader ir = iw.getReader();
     assertEquals(3, ir.numDocs());
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestBackwardsCompatibility.java b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestBackwardsCompatibility.java
index d62040f..05142b8 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestBackwardsCompatibility.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestBackwardsCompatibility.java
@@ -51,6 +51,11 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
 
   public static final String oldTaxonomyIndexName = "taxonomy.8.6.3-cfs";
 
+  // LUCENE-9334 requires consistency of field data structures between documents.
+  // Old taxonomy index had $full_path$ field indexed only with postings,
+  // It is not allowed to add the same field $full_path$ indexed with BinaryDocValues
+  // for a new segment, that this test is trying to do.
+  @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/LUCENE-9334")
   public void testCreateNewTaxonomy() throws IOException {
     createNewTaxonomyIndex(oldTaxonomyIndexName);
   }
diff --git a/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestDistinctValuesCollector.java b/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestDistinctValuesCollector.java
index 8ec8e88..d8fe9e6 100644
--- a/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestDistinctValuesCollector.java
+++ b/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestDistinctValuesCollector.java
@@ -112,7 +112,7 @@ public class TestDistinctValuesCollector extends AbstractGroupingTestCase {
 
     // 6 -- no author field
     doc = new Document();
-    doc.add(new TextField("content", "random word stuck in alot of other text", Field.Store.YES));
+    doc.add(new TextField("content", "random word stuck in alot of other text", Field.Store.NO));
     addField(doc, COUNT_FIELD, "1");
     doc.add(new StringField("id", "6", Field.Store.NO));
     w.addDocument(doc);
diff --git a/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java b/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
index 3cbf735..1e01341 100644
--- a/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
+++ b/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
@@ -740,6 +740,7 @@ public class TestGrouping extends LuceneTestCase {
 
       Field group = newStringField("group", "", Field.Store.NO);
       doc.add(group);
+      docNoGroup.add(group);
       Field sort1 = new SortedDocValuesField("sort1", new BytesRef());
       doc.add(sort1);
       docNoGroup.add(sort1);
diff --git a/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java b/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java
index 82886d3..5f13d59 100644
--- a/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java
+++ b/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java
@@ -86,6 +86,7 @@ public class TestBlockJoin extends LuceneTestCase {
     Document job = new Document();
     job.add(newStringField("qualification", qualification, Field.Store.YES));
     job.add(new IntPoint("year", year));
+    job.add(new StoredField("year", year));
     return job;
   }
 
diff --git a/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinScorer.java b/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinScorer.java
index da1c6e6..4e220db 100644
--- a/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinScorer.java
+++ b/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinScorer.java
@@ -58,7 +58,7 @@ public class TestBlockJoinScorer extends LuceneTestCase {
       }
       Document parent = new Document();
       parent.add(newStringField("docType", "parent", Field.Store.NO));
-      parent.add(newStringField("value", Integer.toString(i), Field.Store.NO));
+      parent.add(newStringField("value", Integer.toString(i), Field.Store.YES));
       docs.add(parent);
       w.addDocuments(docs);
     }
diff --git a/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java b/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
index 6180587..f5431af 100644
--- a/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
+++ b/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
@@ -848,6 +848,7 @@ public class TestJoinUtil extends LuceneTestCase {
     doc.add(new TextField("description", "more random text", Field.Store.NO));
     doc.add(new TextField("name", "name2", Field.Store.NO));
     doc.add(new TextField(idField, "0", Field.Store.NO));
+    doc.add(new SortedDocValuesField(idField, new BytesRef("0")));
     w.addDocument(doc);
     w.commit();
 
diff --git a/lucene/luke/src/test/org/apache/lucene/luke/models/overview/OverviewTestBase.java b/lucene/luke/src/test/org/apache/lucene/luke/models/overview/OverviewTestBase.java
index 79fd0b6..25be932 100644
--- a/lucene/luke/src/test/org/apache/lucene/luke/models/overview/OverviewTestBase.java
+++ b/lucene/luke/src/test/org/apache/lucene/luke/models/overview/OverviewTestBase.java
@@ -62,7 +62,7 @@ public abstract class OverviewTestBase extends LuceneTestCase {
 
     Document doc1 = new Document();
     doc1.add(newStringField("f1", "1", Field.Store.NO));
-    doc1.add(newTextField("f2", "a b c d e", Field.Store.NO));
+    doc1.add(new TextField("f2", "a b c d e", Field.Store.NO));
     writer.addDocument(doc1);
 
     Document doc2 = new Document();
@@ -72,7 +72,7 @@ public abstract class OverviewTestBase extends LuceneTestCase {
 
     Document doc3 = new Document();
     doc3.add(newStringField("f1", "3", Field.Store.NO));
-    doc3.add(newTextField("f2", "a f", Field.Store.NO));
+    doc3.add(new TextField("f2", "a f", Field.Store.NO));
     writer.addDocument(doc3);
 
     Map<String, String> userData = new HashMap<>();
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestDocumentDictionary.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestDocumentDictionary.java
index 12dec50..f20285f 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestDocumentDictionary.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestDocumentDictionary.java
@@ -366,14 +366,13 @@ public class TestDocumentDictionary extends LuceneTestCase {
 
       // usually have valid weight field in document
       if (usually()) {
-        Field weight =
-            (rarely())
-                ? new StoredField(WEIGHT_FIELD_NAME, 100d + i)
-                : new NumericDocValuesField(WEIGHT_FIELD_NAME, 100 + i);
+        Field weight = new StoredField(WEIGHT_FIELD_NAME, 100d + i);
+        doc.add(weight);
+        weight = new NumericDocValuesField(WEIGHT_FIELD_NAME, 100 + i);
         doc.add(weight);
       }
 
-      String term = null;
+      String term;
       if (invalidDoc) {
         term = (field != null) ? field.stringValue() : "invalid_" + i;
         invalidDocTerms.add(term);
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseFieldInfoFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseFieldInfoFormatTestCase.java
index 7e156a3..f635f6d 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseFieldInfoFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseFieldInfoFormatTestCase.java
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
 import com.carrotsearch.randomizedtesting.generators.RandomPicks;
 import java.io.IOException;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Random;
@@ -50,12 +51,13 @@ public abstract class BaseFieldInfoFormatTestCase extends BaseIndexFileFormatTes
     Directory dir = newDirectory();
     Codec codec = getCodec();
     SegmentInfo segmentInfo = newSegmentInfo(dir, "_123");
-    FieldInfos.Builder builder = new FieldInfos.Builder(new FieldInfos.FieldNumbers(null));
-    FieldInfo fi = builder.getOrAdd("field");
-    fi.setIndexOptions(TextField.TYPE_STORED.indexOptions());
+    FieldInfo fi = createFieldInfo();
     addAttributes(fi);
+    FieldInfos.Builder builder = new FieldInfos.Builder(new FieldInfos.FieldNumbers(null));
+    builder.add(fi);
     FieldInfos infos = builder.finish();
     codec.fieldInfosFormat().write(dir, segmentInfo, "", infos, IOContext.DEFAULT);
+
     FieldInfos infos2 = codec.fieldInfosFormat().read(dir, segmentInfo, "", IOContext.DEFAULT);
     assertEquals(1, infos2.size());
     assertNotNull(infos2.fieldInfo("field"));
@@ -75,14 +77,15 @@ public abstract class BaseFieldInfoFormatTestCase extends BaseIndexFileFormatTes
     Directory dir = newDirectory();
     Codec codec = getCodec();
     SegmentInfo segmentInfo = newSegmentInfo(dir, "_123");
-    FieldInfos.Builder builder = new FieldInfos.Builder(new FieldInfos.FieldNumbers(null));
-    FieldInfo fi = builder.getOrAdd("field");
-    fi.setIndexOptions(TextField.TYPE_STORED.indexOptions());
+    FieldInfo fi = createFieldInfo();
     addAttributes(fi);
     fi.putAttribute("foo", "bar");
     fi.putAttribute("bar", "baz");
+    FieldInfos.Builder builder = new FieldInfos.Builder(new FieldInfos.FieldNumbers(null));
+    builder.add(fi);
     FieldInfos infos = builder.finish();
     codec.fieldInfosFormat().write(dir, segmentInfo, "", infos, IOContext.DEFAULT);
+
     FieldInfos infos2 = codec.fieldInfosFormat().read(dir, segmentInfo, "", IOContext.DEFAULT);
     assertEquals(1, infos2.size());
     assertNotNull(infos2.fieldInfo("field"));
@@ -116,10 +119,10 @@ public abstract class BaseFieldInfoFormatTestCase extends BaseIndexFileFormatTes
     dir.failOn(fail);
     Codec codec = getCodec();
     SegmentInfo segmentInfo = newSegmentInfo(dir, "_123");
-    FieldInfos.Builder builder = new FieldInfos.Builder(new FieldInfos.FieldNumbers(null));
-    FieldInfo fi = builder.getOrAdd("field");
-    fi.setIndexOptions(TextField.TYPE_STORED.indexOptions());
+    FieldInfo fi = createFieldInfo();
     addAttributes(fi);
+    FieldInfos.Builder builder = new FieldInfos.Builder(new FieldInfos.FieldNumbers(null));
+    builder.add(fi);
     FieldInfos infos = builder.finish();
 
     fail.setDoFail();
@@ -152,9 +155,10 @@ public abstract class BaseFieldInfoFormatTestCase extends BaseIndexFileFormatTes
     dir.failOn(fail);
     Codec codec = getCodec();
     SegmentInfo segmentInfo = newSegmentInfo(dir, "_123");
+    FieldInfo fi = createFieldInfo();
+
     FieldInfos.Builder builder = new FieldInfos.Builder(new FieldInfos.FieldNumbers(null));
-    FieldInfo fi = builder.getOrAdd("field");
-    fi.setIndexOptions(TextField.TYPE_STORED.indexOptions());
+    builder.add(fi);
     addAttributes(fi);
     FieldInfos infos = builder.finish();
 
@@ -188,9 +192,10 @@ public abstract class BaseFieldInfoFormatTestCase extends BaseIndexFileFormatTes
     dir.failOn(fail);
     Codec codec = getCodec();
     SegmentInfo segmentInfo = newSegmentInfo(dir, "_123");
+    FieldInfo fi = createFieldInfo();
+
     FieldInfos.Builder builder = new FieldInfos.Builder(new FieldInfos.FieldNumbers(null));
-    FieldInfo fi = builder.getOrAdd("field");
-    fi.setIndexOptions(TextField.TYPE_STORED.indexOptions());
+    builder.add(fi);
     addAttributes(fi);
     FieldInfos infos = builder.finish();
     codec.fieldInfosFormat().write(dir, segmentInfo, "", infos, IOContext.DEFAULT);
@@ -225,9 +230,9 @@ public abstract class BaseFieldInfoFormatTestCase extends BaseIndexFileFormatTes
     dir.failOn(fail);
     Codec codec = getCodec();
     SegmentInfo segmentInfo = newSegmentInfo(dir, "_123");
+    FieldInfo fi = createFieldInfo();
     FieldInfos.Builder builder = new FieldInfos.Builder(new FieldInfos.FieldNumbers(null));
-    FieldInfo fi = builder.getOrAdd("field");
-    fi.setIndexOptions(TextField.TYPE_STORED.indexOptions());
+    builder.add(fi);
     addAttributes(fi);
     FieldInfos infos = builder.finish();
     codec.fieldInfosFormat().write(dir, segmentInfo, "", infos, IOContext.DEFAULT);
@@ -265,35 +270,35 @@ public abstract class BaseFieldInfoFormatTestCase extends BaseIndexFileFormatTes
 
     for (String field : fieldNames) {
       IndexableFieldType fieldType = randomFieldType(random());
-      FieldInfo fi = builder.getOrAdd(field);
-      IndexOptions indexOptions = fieldType.indexOptions();
-      if (indexOptions != IndexOptions.NONE) {
-        fi.setIndexOptions(indexOptions);
-        if (fieldType.omitNorms()) {
-          fi.setOmitsNorms();
-        }
-      }
-      fi.setDocValuesType(fieldType.docValuesType());
-      if (fieldType.indexOptions() != IndexOptions.NONE
-          && fieldType.indexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0) {
-        if (random().nextBoolean()) {
-          fi.setStorePayloads();
+      boolean storeTermVectors = false;
+      boolean storePayloads = false;
+      boolean omitNorms = false;
+      if (fieldType.indexOptions() != IndexOptions.NONE) {
+        storeTermVectors = fieldType.storeTermVectors();
+        omitNorms = fieldType.omitNorms();
+        if (fieldType.indexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0) {
+          storePayloads = random().nextBoolean();
         }
       }
-
-      if (fieldType.pointDimensionCount() > 0) {
-        fi.setPointDimensions(
-            fieldType.pointDimensionCount(),
-            fieldType.pointIndexDimensionCount(),
-            fieldType.pointNumBytes());
-      }
-
-      if (fieldType.vectorDimension() > 0) {
-        fi.setVectorDimensionAndSearchStrategy(
-            fieldType.vectorDimension(), fieldType.vectorSearchStrategy());
-      }
-
+      FieldInfo fi =
+          new FieldInfo(
+              field,
+              -1,
+              storeTermVectors,
+              omitNorms,
+              storePayloads,
+              fieldType.indexOptions(),
+              fieldType.docValuesType(),
+              -1,
+              new HashMap<>(),
+              fieldType.pointDimensionCount(),
+              fieldType.pointIndexDimensionCount(),
+              fieldType.pointNumBytes(),
+              fieldType.vectorDimension(),
+              fieldType.vectorSearchStrategy(),
+              field.equals(softDeletesField));
       addAttributes(fi);
+      builder.add(fi);
     }
     FieldInfos infos = builder.finish();
     codec.fieldInfosFormat().write(dir, segmentInfo, "", infos, IOContext.DEFAULT);
@@ -391,4 +396,23 @@ public abstract class BaseFieldInfoFormatTestCase extends BaseIndexFileFormatTes
   protected void addRandomFields(Document doc) {
     doc.add(new StoredField("foobar", TestUtil.randomSimpleString(random())));
   }
+
+  private FieldInfo createFieldInfo() {
+    return new FieldInfo(
+        "field",
+        -1,
+        false,
+        false,
+        false,
+        TextField.TYPE_STORED.indexOptions(),
+        DocValuesType.NONE,
+        -1,
+        new HashMap<>(),
+        0,
+        0,
+        0,
+        0,
+        VectorValues.SearchStrategy.NONE,
+        false);
+  }
 }
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BasePointsFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BasePointsFormatTestCase.java
index d489e4b..bd7a757 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BasePointsFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BasePointsFormatTestCase.java
@@ -1164,25 +1164,22 @@ public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCa
   }
 
   // LUCENE-7491
-  public void testMixedSchema() throws Exception {
+  public void testMergeMissing() throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
     RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
     iwc.setMaxBufferedDocs(2);
     for (int i = 0; i < 2; i++) {
       Document doc = new Document();
-      doc.add(new StringField("id", Integer.toString(i), Field.Store.NO));
       doc.add(new IntPoint("int", i));
       w.addDocument(doc);
     }
-    // index has 1 segment now (with 2 docs) and that segment does have points, but the "id" field
-    // in particular does NOT
+    // index has 1 segment now (with 2 docs) and that segment does have points
 
     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:
-
     w.forceMerge(1);
     IOUtils.close(w, dir);
   }
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseVectorFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseVectorFormatTestCase.java
index 047c373..a17c6d7 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseVectorFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseVectorFormatTestCase.java
@@ -52,22 +52,6 @@ public abstract class BaseVectorFormatTestCase extends BaseIndexFileFormatTestCa
     doc.add(new VectorField("v2", randomVector(30), VectorValues.SearchStrategy.NONE));
   }
 
-  // Suddenly add vectors to an existing field:
-  public void testUpgradeFieldToVectors() throws Exception {
-    try (Directory dir = newDirectory()) {
-      try (IndexWriter w = new IndexWriter(dir, newIndexWriterConfig())) {
-        Document doc = new Document();
-        doc.add(newStringField("f", "foo", Field.Store.NO));
-        w.addDocument(doc);
-      }
-      try (IndexWriter w = new IndexWriter(dir, newIndexWriterConfig())) {
-        Document doc = new Document();
-        doc.add(new VectorField("f", new float[4], VectorValues.SearchStrategy.DOT_PRODUCT_HNSW));
-        w.addDocument(doc);
-      }
-    }
-  }
-
   public void testFieldConstructor() {
     float[] v = new float[1];
     VectorField field = new VectorField("f", v);
@@ -124,45 +108,75 @@ public abstract class BaseVectorFormatTestCase extends BaseIndexFileFormatTestCa
   }
 
   // Illegal schema change tests:
-
   public void testIllegalDimChangeTwoDocs() throws Exception {
+    // illegal change in the same segment
     try (Directory dir = newDirectory();
         IndexWriter w = new IndexWriter(dir, newIndexWriterConfig())) {
       Document doc = new Document();
       doc.add(new VectorField("f", new float[4], VectorValues.SearchStrategy.DOT_PRODUCT_HNSW));
       w.addDocument(doc);
-      if (random().nextBoolean()) {
-        // sometimes test with two segments
-        w.commit();
-      }
 
       Document doc2 = new Document();
       doc2.add(new VectorField("f", new float[3], VectorValues.SearchStrategy.DOT_PRODUCT_HNSW));
       IllegalArgumentException expected =
           expectThrows(IllegalArgumentException.class, () -> w.addDocument(doc2));
-      assertEquals(
-          "cannot change vector dimension from 4 to 3 for field=\"f\"", expected.getMessage());
+      String errMsg =
+          "Inconsistency of field data structures across documents for field [f] of doc [1].";
+      assertEquals(errMsg, expected.getMessage());
+    }
+
+    // illegal change in a different segment
+    try (Directory dir = newDirectory();
+        IndexWriter w = new IndexWriter(dir, newIndexWriterConfig())) {
+      Document doc = new Document();
+      doc.add(new VectorField("f", new float[4], VectorValues.SearchStrategy.DOT_PRODUCT_HNSW));
+      w.addDocument(doc);
+      w.commit();
+
+      Document doc2 = new Document();
+      doc2.add(new VectorField("f", new float[3], VectorValues.SearchStrategy.DOT_PRODUCT_HNSW));
+      IllegalArgumentException expected =
+          expectThrows(IllegalArgumentException.class, () -> w.addDocument(doc2));
+      String errMsg =
+          "cannot change field \"f\" from vector dimension=4, vector search strategy=DOT_PRODUCT_HNSW "
+              + "to inconsistent vector dimension=3, vector search strategy=DOT_PRODUCT_HNSW";
+      assertEquals(errMsg, expected.getMessage());
     }
   }
 
   public void testIllegalSearchStrategyChange() throws Exception {
+    // illegal change in the same segment
     try (Directory dir = newDirectory();
         IndexWriter w = new IndexWriter(dir, newIndexWriterConfig())) {
       Document doc = new Document();
       doc.add(new VectorField("f", new float[4], VectorValues.SearchStrategy.DOT_PRODUCT_HNSW));
       w.addDocument(doc);
-      if (random().nextBoolean()) {
-        // sometimes test with two segments
-        w.commit();
-      }
 
       Document doc2 = new Document();
       doc2.add(new VectorField("f", new float[4], VectorValues.SearchStrategy.EUCLIDEAN_HNSW));
       IllegalArgumentException expected =
           expectThrows(IllegalArgumentException.class, () -> w.addDocument(doc2));
-      assertEquals(
-          "cannot change vector search strategy from DOT_PRODUCT_HNSW to EUCLIDEAN_HNSW for field=\"f\"",
-          expected.getMessage());
+      String errMsg =
+          "Inconsistency of field data structures across documents for field [f] of doc [1].";
+      assertEquals(errMsg, expected.getMessage());
+    }
+
+    // illegal change a different segment
+    try (Directory dir = newDirectory();
+        IndexWriter w = new IndexWriter(dir, newIndexWriterConfig())) {
+      Document doc = new Document();
+      doc.add(new VectorField("f", new float[4], VectorValues.SearchStrategy.DOT_PRODUCT_HNSW));
+      w.addDocument(doc);
+      w.commit();
+
+      Document doc2 = new Document();
+      doc2.add(new VectorField("f", new float[4], VectorValues.SearchStrategy.EUCLIDEAN_HNSW));
+      IllegalArgumentException expected =
+          expectThrows(IllegalArgumentException.class, () -> w.addDocument(doc2));
+      String errMsg =
+          "cannot change field \"f\" from vector dimension=4, vector search strategy=DOT_PRODUCT_HNSW "
+              + "to inconsistent vector dimension=4, vector search strategy=EUCLIDEAN_HNSW";
+      assertEquals(errMsg, expected.getMessage());
     }
   }
 
@@ -180,7 +194,9 @@ public abstract class BaseVectorFormatTestCase extends BaseIndexFileFormatTestCa
         IllegalArgumentException expected =
             expectThrows(IllegalArgumentException.class, () -> w2.addDocument(doc2));
         assertEquals(
-            "cannot change vector dimension from 4 to 1 for field=\"f\"", expected.getMessage());
+            "cannot change field \"f\" from vector dimension=4, vector search strategy=DOT_PRODUCT_HNSW "
+                + "to inconsistent vector dimension=1, vector search strategy=DOT_PRODUCT_HNSW",
+            expected.getMessage());
       }
     }
   }
@@ -199,7 +215,8 @@ public abstract class BaseVectorFormatTestCase extends BaseIndexFileFormatTestCa
         IllegalArgumentException expected =
             expectThrows(IllegalArgumentException.class, () -> w2.addDocument(doc2));
         assertEquals(
-            "cannot change vector search strategy from DOT_PRODUCT_HNSW to EUCLIDEAN_HNSW for field=\"f\"",
+            "cannot change field \"f\" from vector dimension=4, vector search strategy=DOT_PRODUCT_HNSW "
+                + "to inconsistent vector dimension=4, vector search strategy=EUCLIDEAN_HNSW",
             expected.getMessage());
       }
     }
@@ -299,7 +316,9 @@ public abstract class BaseVectorFormatTestCase extends BaseIndexFileFormatTestCa
             expectThrows(
                 IllegalArgumentException.class, () -> w2.addIndexes(new Directory[] {dir}));
         assertEquals(
-            "cannot change vector dimension from 5 to 4 for field=\"f\"", expected.getMessage());
+            "cannot change field \"f\" from vector dimension=5, vector search strategy=DOT_PRODUCT_HNSW "
+                + "to inconsistent vector dimension=4, vector search strategy=DOT_PRODUCT_HNSW",
+            expected.getMessage());
       }
     }
   }
@@ -319,7 +338,8 @@ public abstract class BaseVectorFormatTestCase extends BaseIndexFileFormatTestCa
         IllegalArgumentException expected =
             expectThrows(IllegalArgumentException.class, () -> w2.addIndexes(dir));
         assertEquals(
-            "cannot change vector search strategy from EUCLIDEAN_HNSW to DOT_PRODUCT_HNSW for field=\"f\"",
+            "cannot change field \"f\" from vector dimension=4, vector search strategy=EUCLIDEAN_HNSW "
+                + "to inconsistent vector dimension=4, vector search strategy=DOT_PRODUCT_HNSW",
             expected.getMessage());
       }
     }
@@ -343,7 +363,9 @@ public abstract class BaseVectorFormatTestCase extends BaseIndexFileFormatTestCa
                   IllegalArgumentException.class,
                   () -> w2.addIndexes(new CodecReader[] {(CodecReader) getOnlyLeafReader(r)}));
           assertEquals(
-              "cannot change vector dimension from 5 to 4 for field=\"f\"", expected.getMessage());
+              "cannot change field \"f\" from vector dimension=5, vector search strategy=DOT_PRODUCT_HNSW "
+                  + "to inconsistent vector dimension=4, vector search strategy=DOT_PRODUCT_HNSW",
+              expected.getMessage());
         }
       }
     }
@@ -367,7 +389,8 @@ public abstract class BaseVectorFormatTestCase extends BaseIndexFileFormatTestCa
                   IllegalArgumentException.class,
                   () -> w2.addIndexes(new CodecReader[] {(CodecReader) getOnlyLeafReader(r)}));
           assertEquals(
-              "cannot change vector search strategy from EUCLIDEAN_HNSW to DOT_PRODUCT_HNSW for field=\"f\"",
+              "cannot change field \"f\" from vector dimension=4, vector search strategy=EUCLIDEAN_HNSW "
+                  + "to inconsistent vector dimension=4, vector search strategy=DOT_PRODUCT_HNSW",
               expected.getMessage());
         }
       }
@@ -390,7 +413,9 @@ public abstract class BaseVectorFormatTestCase extends BaseIndexFileFormatTestCa
           IllegalArgumentException expected =
               expectThrows(IllegalArgumentException.class, () -> TestUtil.addIndexesSlowly(w2, r));
           assertEquals(
-              "cannot change vector dimension from 5 to 4 for field=\"f\"", expected.getMessage());
+              "cannot change field \"f\" from vector dimension=5, vector search strategy=DOT_PRODUCT_HNSW "
+                  + "to inconsistent vector dimension=4, vector search strategy=DOT_PRODUCT_HNSW",
+              expected.getMessage());
         }
       }
     }
@@ -412,7 +437,8 @@ public abstract class BaseVectorFormatTestCase extends BaseIndexFileFormatTestCa
           IllegalArgumentException expected =
               expectThrows(IllegalArgumentException.class, () -> TestUtil.addIndexesSlowly(w2, r));
           assertEquals(
-              "cannot change vector search strategy from EUCLIDEAN_HNSW to DOT_PRODUCT_HNSW for field=\"f\"",
+              "cannot change field \"f\" from vector dimension=4, vector search strategy=EUCLIDEAN_HNSW "
+                  + "to inconsistent vector dimension=4, vector search strategy=DOT_PRODUCT_HNSW",
               expected.getMessage());
         }
       }
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/DocHelper.java b/lucene/test-framework/src/java/org/apache/lucene/index/DocHelper.java
index dd26a59..84c6213 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/DocHelper.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/DocHelper.java
@@ -47,7 +47,7 @@ public class DocHelper {
     textField1 = new Field(TEXT_FIELD_1_KEY, FIELD_1_TEXT, customType);
   }
 
-  public static final FieldType customType2;
+  public static final FieldType TEXT_TYPE_STORED_WITH_TVS;
   public static final String FIELD_2_TEXT = "field field field two text";
   // Fields will be lexicographically sorted.  So, the order is: field, text, two
   public static final int[] FIELD_2_FREQS = {3, 1, 1};
@@ -55,11 +55,12 @@ public class DocHelper {
   public static Field textField2;
 
   static {
-    customType2 = new FieldType(TextField.TYPE_STORED);
-    customType2.setStoreTermVectors(true);
-    customType2.setStoreTermVectorPositions(true);
-    customType2.setStoreTermVectorOffsets(true);
-    textField2 = new Field(TEXT_FIELD_2_KEY, FIELD_2_TEXT, customType2);
+    TEXT_TYPE_STORED_WITH_TVS = new FieldType(TextField.TYPE_STORED);
+    TEXT_TYPE_STORED_WITH_TVS.setStoreTermVectors(true);
+    TEXT_TYPE_STORED_WITH_TVS.setStoreTermVectorPositions(true);
+    TEXT_TYPE_STORED_WITH_TVS.setStoreTermVectorOffsets(true);
+    TEXT_TYPE_STORED_WITH_TVS.freeze();
+    textField2 = new Field(TEXT_FIELD_2_KEY, FIELD_2_TEXT, TEXT_TYPE_STORED_WITH_TVS);
   }
 
   public static final FieldType customType3;
@@ -115,6 +116,16 @@ public class DocHelper {
     unIndField = new Field(UNINDEXED_FIELD_KEY, UNINDEXED_FIELD_TEXT, customType7);
   }
 
+  public static final FieldType STRING_TYPE_STORED_WITH_TVS;
+
+  static {
+    STRING_TYPE_STORED_WITH_TVS = new FieldType(StringField.TYPE_STORED);
+    STRING_TYPE_STORED_WITH_TVS.setStoreTermVectors(true);
+    STRING_TYPE_STORED_WITH_TVS.setStoreTermVectorPositions(true);
+    STRING_TYPE_STORED_WITH_TVS.setStoreTermVectorOffsets(true);
+    STRING_TYPE_STORED_WITH_TVS.freeze();
+  }
+
   public static final String UNSTORED_1_FIELD_TEXT = "unstored field text";
   public static final String UNSTORED_FIELD_1_KEY = "unStoredField1";
   public static Field unStoredField1 =
@@ -152,7 +163,8 @@ public class DocHelper {
   // Fields will be lexicographically sorted.  So, the order is: field, text, two
   public static final int[] FIELD_UTF2_FREQS = {3, 1, 1};
   public static final String TEXT_FIELD_UTF2_KEY = "textField2Utf8";
-  public static Field textUtfField2 = new Field(TEXT_FIELD_UTF2_KEY, FIELD_UTF2_TEXT, customType2);
+  public static Field textUtfField2 =
+      new Field(TEXT_FIELD_UTF2_KEY, FIELD_UTF2_TEXT, TEXT_TYPE_STORED_WITH_TVS);
 
   public static Map<String, Object> nameValues = null;
 
@@ -291,26 +303,17 @@ public class DocHelper {
 
   public static Document createDocument(int n, String indexName, int numFields) {
     StringBuilder sb = new StringBuilder();
-    FieldType customType = new FieldType(TextField.TYPE_STORED);
-    customType.setStoreTermVectors(true);
-    customType.setStoreTermVectorPositions(true);
-    customType.setStoreTermVectorOffsets(true);
-
-    FieldType customType1 = new FieldType(StringField.TYPE_STORED);
-    customType1.setStoreTermVectors(true);
-    customType1.setStoreTermVectorPositions(true);
-    customType1.setStoreTermVectorOffsets(true);
 
     final Document doc = new Document();
-    doc.add(new Field("id", Integer.toString(n), customType1));
-    doc.add(new Field("indexname", indexName, customType1));
+    doc.add(new Field("id", Integer.toString(n), STRING_TYPE_STORED_WITH_TVS));
+    doc.add(new Field("indexname", indexName, STRING_TYPE_STORED_WITH_TVS));
     sb.append("a");
     sb.append(n);
-    doc.add(new Field("field1", sb.toString(), customType));
+    doc.add(new Field("field1", sb.toString(), TEXT_TYPE_STORED_WITH_TVS));
     sb.append(" b");
     sb.append(n);
     for (int i = 1; i < numFields; i++) {
-      doc.add(new Field("field" + (i + 1), sb.toString(), customType));
+      doc.add(new Field("field" + (i + 1), sb.toString(), TEXT_TYPE_STORED_WITH_TVS));
     }
     return doc;
   }
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
index 12cfb38..450d002 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
@@ -1559,22 +1559,6 @@ public abstract class LuceneTestCase extends Assert {
     return newField(random(), name, value, type);
   }
 
-  /** Returns a FieldType derived from newType but whose term vector options match the old type */
-  private static FieldType mergeTermVectorOptions(FieldType newType, FieldType oldType) {
-    if (newType.indexOptions() != IndexOptions.NONE
-        && oldType.storeTermVectors() == true
-        && newType.storeTermVectors() == false) {
-      newType = new FieldType(newType);
-      newType.setStoreTermVectors(oldType.storeTermVectors());
-      newType.setStoreTermVectorPositions(oldType.storeTermVectorPositions());
-      newType.setStoreTermVectorOffsets(oldType.storeTermVectorOffsets());
-      newType.setStoreTermVectorPayloads(oldType.storeTermVectorPayloads());
-      newType.freeze();
-    }
-
-    return newType;
-  }
-
   // TODO: if we can pull out the "make term vector options
   // consistent across all instances of the same field name"
   // write-once schema sort of helper class then we can
@@ -1588,16 +1572,9 @@ public abstract class LuceneTestCase extends Assert {
     name = new String(name);
 
     FieldType prevType = fieldToType.get(name);
-
-    if (usually(random) || type.indexOptions() == IndexOptions.NONE || prevType != null) {
-      // most of the time, don't modify the params
-      if (prevType == null) {
-        fieldToType.put(name, new FieldType(type));
-      } else {
-        type = mergeTermVectorOptions(type, prevType);
-      }
-
-      return createField(name, value, type);
+    if (prevType != null) {
+      // always use the same fieldType for the same field name
+      return createField(name, value, prevType);
     }
 
     // TODO: once all core & test codecs can index
@@ -1608,27 +1585,24 @@ public abstract class LuceneTestCase extends Assert {
     if (!newType.stored() && random.nextBoolean()) {
       newType.setStored(true); // randomly store it
     }
-
-    // Randomly turn on term vector options, but always do
-    // so consistently for the same field name:
-    if (!newType.storeTermVectors() && random.nextBoolean()) {
-      newType.setStoreTermVectors(true);
-      if (!newType.storeTermVectorPositions()) {
-        newType.setStoreTermVectorPositions(random.nextBoolean());
-
-        if (newType.storeTermVectorPositions()) {
-          if (!newType.storeTermVectorPayloads()) {
-            newType.setStoreTermVectorPayloads(random.nextBoolean());
+    if (newType.indexOptions() != IndexOptions.NONE) {
+      if (!newType.storeTermVectors() && random.nextBoolean()) {
+        newType.setStoreTermVectors(true);
+        if (!newType.storeTermVectorPositions()) {
+          newType.setStoreTermVectorPositions(random.nextBoolean());
+          if (newType.storeTermVectorPositions()) {
+            if (!newType.storeTermVectorPayloads()) {
+              newType.setStoreTermVectorPayloads(random.nextBoolean());
+            }
           }
         }
-      }
-
-      if (!newType.storeTermVectorOffsets()) {
-        newType.setStoreTermVectorOffsets(random.nextBoolean());
-      }
+        if (!newType.storeTermVectorOffsets()) {
+          newType.setStoreTermVectorOffsets(random.nextBoolean());
+        }
 
-      if (VERBOSE) {
-        System.out.println("NOTE: LuceneTestCase: upgrade name=" + name + " type=" + newType);
+        if (VERBOSE) {
+          System.out.println("NOTE: LuceneTestCase: upgrade name=" + name + " type=" + newType);
+        }
       }
     }
     newType.freeze();