You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2023/01/12 08:32:20 UTC

[lucene] branch main updated: Allow reusing indexed binary fields. (#12053)

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

jpountz 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 84778549afa Allow reusing indexed binary fields. (#12053)
84778549afa is described below

commit 84778549afabc32cd6e5a3459b1b8aaa89b74a74
Author: Adrien Grand <jp...@gmail.com>
AuthorDate: Thu Jan 12 09:32:13 2023 +0100

    Allow reusing indexed binary fields. (#12053)
    
    Today Lucene allows creating indexed binary fields, e.g. via
    `StringField(String, BytesRef, Field.Store)`, but not reusing them: calling
    `setBytesValue` on a `StringField` throws.
    
    This commit removes the check that prevents reusing fields with binary values.
    I considered an alternative that consisted of failing if calling
    `setBytesValue` on a field that is indexed and tokenized, but we currently
    don't have such checks e.g. on numeric values, so it did not feel consistent.
    
    Doing this change would help improve the [nightly benchmarks for the NYC taxis
    dataset](http://people.apache.org/~mikemccand/lucenebench/sparseResults.html)
    by doing the String -> UTF-8 conversion only once for keywords, instead of once
    for the `StringField` and one for the `SortedDocValuesField`, while still
    reusing fields.
---
 lucene/MIGRATE.md                                  |  9 ++++
 .../miscellaneous/TestFixBrokenOffsetsFilter.java  | 10 ++--
 .../src/java/org/apache/lucene/document/Field.java | 54 +++++++++++---------
 .../test/org/apache/lucene/document/TestField.java | 58 +++++++++++++++++++++-
 .../apache/lucene/index/TestDocValuesIndexing.java | 18 ++++---
 .../org/apache/lucene/index/TestIndexWriter.java   | 43 ++++++++--------
 .../lucene/index/TestIndexWriterExceptions.java    | 12 ++---
 .../test/org/apache/lucene/index/TestPayloads.java | 11 ++--
 .../apache/lucene/index/TestPayloadsOnVectors.java | 12 ++---
 .../test/org/apache/lucene/index/TestTerms.java    |  6 +--
 10 files changed, 152 insertions(+), 81 deletions(-)

diff --git a/lucene/MIGRATE.md b/lucene/MIGRATE.md
index 7c00aa1ad7a..e731cfd84ab 100644
--- a/lucene/MIGRATE.md
+++ b/lucene/MIGRATE.md
@@ -38,6 +38,15 @@ for (ScoreDoc hit : hits.scoreDocs) {
 Note that these StoredFields and TermVectors instances should only be consumed in the thread where
 they were acquired. For instance, it is illegal to share them across threads.
 
+### Field can no longer configure a TokenStream independently from a value
+
+Lucene 9.x and earlier versions allowed to set a TokenStream on Field instances
+independently from a string, binary or numeric value. This is no longer allowed
+on the base Field class. If you need to replicate this behavior, you need to
+either provide two fields, one with a TokenStream and another one with a value,
+or create a sub-class of Field that overrides `TokenStream
+tokenStream(Analyzer, TokenStream)` to return a custom TokenStream.
+
 ### PersianStemFilter is added to PersianAnalyzer (LUCENE-10312)
 
 PersianAnalyzer now includes PersianStemFilter, that would change analysis results. If you need the exactly same analysis
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestFixBrokenOffsetsFilter.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestFixBrokenOffsetsFilter.java
index 4b9c8e17424..a670c05143f 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestFixBrokenOffsetsFilter.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestFixBrokenOffsetsFilter.java
@@ -38,10 +38,12 @@ public class TestFixBrokenOffsetsFilter extends BaseTokenStreamTestCase {
     FieldType ft = new FieldType(TextField.TYPE_NOT_STORED);
     ft.setStoreTermVectors(true);
     ft.setStoreTermVectorOffsets(true);
-    Field field = new Field("foo", "", ft);
-    field.setTokenStream(
-        new FixBrokenOffsetsFilter(
-            new CannedTokenStream(new Token("bar", 5, 10), new Token("bar", 1, 4))));
+    Field field =
+        new Field(
+            "foo",
+            new FixBrokenOffsetsFilter(
+                new CannedTokenStream(new Token("bar", 5, 10), new Token("bar", 1, 4))),
+            ft);
     doc.add(field);
     iw.addDocument(doc);
     iw.close();
diff --git a/lucene/core/src/java/org/apache/lucene/document/Field.java b/lucene/core/src/java/org/apache/lucene/document/Field.java
index 126e85e3e22..94e86c7bcdb 100644
--- a/lucene/core/src/java/org/apache/lucene/document/Field.java
+++ b/lucene/core/src/java/org/apache/lucene/document/Field.java
@@ -23,6 +23,7 @@ import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.BytesTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.IndexableFieldType;
@@ -69,13 +70,8 @@ public class Field implements IndexableField {
   protected Object fieldsData;
 
   /**
-   * Pre-analyzed tokenStream for indexed fields; this is separate from fieldsData because you are
-   * allowed to have both; eg maybe field has a String value but you customize how it's tokenized
-   */
-  protected TokenStream tokenStream;
-
-  /**
-   * Expert: creates a field with no initial value. Intended only for custom Field subclasses.
+   * Expert: creates a field with no initial value. This is intended to be used by custom {@link
+   * Field} sub-classes with pre-configured {@link IndexableFieldType}s.
    *
    * @param name field name
    * @param type field type
@@ -149,8 +145,7 @@ public class Field implements IndexableField {
     }
 
     this.name = name;
-    this.fieldsData = null;
-    this.tokenStream = tokenStream;
+    this.fieldsData = tokenStream;
     this.type = type;
   }
 
@@ -210,6 +205,20 @@ public class Field implements IndexableField {
     if (type == null) {
       throw new IllegalArgumentException("type must not be null");
     }
+    if (type.indexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0
+        || type.storeTermVectorOffsets()) {
+      throw new IllegalArgumentException("It doesn't make sense to index offsets on binary fields");
+    }
+    if (type.indexOptions() != IndexOptions.NONE && type.tokenized()) {
+      throw new IllegalArgumentException("cannot set a BytesRef value on a tokenized field");
+    }
+    if (type.indexOptions() == IndexOptions.NONE
+        && type.pointDimensionCount() == 0
+        && type.docValuesType() == DocValuesType.NONE
+        && type.stored() == false) {
+      throw new IllegalArgumentException(
+          "it doesn't make sense to have a field that is neither indexed, nor doc-valued, nor stored");
+    }
     this.name = name;
     this.fieldsData = bytes;
     this.type = type;
@@ -237,9 +246,9 @@ public class Field implements IndexableField {
     if (type == null) {
       throw new IllegalArgumentException("type must not be null");
     }
-    if (!type.stored() && type.indexOptions() == IndexOptions.NONE) {
+    if (type.stored() == false && type.indexOptions() == IndexOptions.NONE) {
       throw new IllegalArgumentException(
-          "it doesn't make sense to have a field that " + "is neither indexed nor stored");
+          "it doesn't make sense to have a field that is neither indexed nor stored");
     }
     this.name = name;
     this.fieldsData = value;
@@ -278,7 +287,7 @@ public class Field implements IndexableField {
    * String value is analyzed to produce the indexed tokens.
    */
   public TokenStream tokenStreamValue() {
-    return tokenStream;
+    return fieldsData instanceof TokenStream ? (TokenStream) fieldsData : null;
   }
 
   /**
@@ -329,9 +338,6 @@ public class Field implements IndexableField {
               + fieldsData.getClass().getSimpleName()
               + " to BytesRef");
     }
-    if (type.indexOptions() != IndexOptions.NONE) {
-      throw new IllegalArgumentException("cannot set a BytesRef value on an indexed field");
-    }
     if (value == null) {
       throw new IllegalArgumentException("value must not be null");
     }
@@ -392,15 +398,15 @@ public class Field implements IndexableField {
     fieldsData = Double.valueOf(value);
   }
 
-  /**
-   * Expert: sets the token stream to be used for indexing and causes isIndexed() and isTokenized()
-   * to return true. May be combined with stored values from stringValue() or binaryValue()
-   */
+  /** Expert: sets the token stream to be used for indexing. */
   public void setTokenStream(TokenStream tokenStream) {
-    if (type.indexOptions() == IndexOptions.NONE || !type.tokenized()) {
-      throw new IllegalArgumentException("TokenStream fields must be indexed and tokenized");
+    if (!(fieldsData instanceof TokenStream)) {
+      throw new IllegalArgumentException(
+          "cannot change value type from "
+              + fieldsData.getClass().getSimpleName()
+              + " to TokenStream");
     }
-    this.tokenStream = tokenStream;
+    this.fieldsData = tokenStream;
   }
 
   @Override
@@ -478,8 +484,8 @@ public class Field implements IndexableField {
       }
     }
 
-    if (tokenStream != null) {
-      return tokenStream;
+    if (tokenStreamValue() != null) {
+      return tokenStreamValue();
     } else if (readerValue() != null) {
       return analyzer.tokenStream(name(), readerValue());
     } else if (stringValue() != null) {
diff --git a/lucene/core/src/test/org/apache/lucene/document/TestField.java b/lucene/core/src/test/org/apache/lucene/document/TestField.java
index 534f06ff7a6..0c61a9d2ea6 100644
--- a/lucene/core/src/test/org/apache/lucene/document/TestField.java
+++ b/lucene/core/src/test/org/apache/lucene/document/TestField.java
@@ -23,6 +23,7 @@ import java.nio.charset.StandardCharsets;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.index.ByteVectorValues;
 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.Term;
@@ -425,6 +426,31 @@ public class TestField extends LuceneTestCase {
     }
   }
 
+  public void testBinaryStringField() throws Exception {
+    Field[] fields =
+        new Field[] {
+          new StringField("foo", new BytesRef("bar"), Field.Store.NO),
+          new StringField("foo", new BytesRef("bar"), Field.Store.YES)
+        };
+
+    for (Field field : fields) {
+      trySetByteValue(field);
+      field.setBytesValue("baz".getBytes(StandardCharsets.UTF_8));
+      assertEquals(new BytesRef("baz"), field.binaryValue());
+      field.setBytesValue(new BytesRef("baz"));
+      trySetDoubleValue(field);
+      trySetIntValue(field);
+      trySetFloatValue(field);
+      trySetLongValue(field);
+      trySetReaderValue(field);
+      trySetShortValue(field);
+      trySetStringValue(field);
+      trySetTokenStreamValue(field);
+
+      assertEquals(new BytesRef("baz"), field.binaryValue());
+    }
+  }
+
   public void testTextFieldString() throws Exception {
     Field[] fields =
         new Field[] {
@@ -442,7 +468,7 @@ public class TestField extends LuceneTestCase {
       trySetReaderValue(field);
       trySetShortValue(field);
       field.setStringValue("baz");
-      field.setTokenStream(new CannedTokenStream(new Token("foo", 0, 3)));
+      trySetTokenStreamValue(field);
 
       assertEquals("baz", field.stringValue());
     }
@@ -461,7 +487,7 @@ public class TestField extends LuceneTestCase {
     field.setReaderValue(new StringReader("foobar"));
     trySetShortValue(field);
     trySetStringValue(field);
-    field.setTokenStream(new CannedTokenStream(new Token("foo", 0, 3)));
+    trySetTokenStreamValue(field);
 
     assertNotNull(field.readerValue());
   }
@@ -728,4 +754,32 @@ public class TestField extends LuceneTestCase {
           f.setTokenStream(new CannedTokenStream(new Token("foo", 0, 3)));
         });
   }
+
+  public void testDisabledField() {
+    // neither indexed nor stored
+    FieldType ft = new FieldType();
+    expectThrows(IllegalArgumentException.class, () -> new Field("name", "", ft));
+  }
+
+  public void testTokenizedBinaryField() {
+    FieldType ft = new FieldType();
+    ft.setTokenized(true);
+    ft.setIndexOptions(IndexOptions.DOCS);
+    expectThrows(IllegalArgumentException.class, () -> new Field("name", new BytesRef(), ft));
+  }
+
+  public void testOffsetsBinaryField() {
+    FieldType ft = new FieldType();
+    ft.setTokenized(false);
+    ft.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+    expectThrows(IllegalArgumentException.class, () -> new Field("name", new BytesRef(), ft));
+  }
+
+  public void testTermVectorsOffsetsBinaryField() {
+    FieldType ft = new FieldType();
+    ft.setTokenized(false);
+    ft.setStoreTermVectors(true);
+    ft.setStoreTermVectorOffsets(true);
+    expectThrows(IllegalArgumentException.class, () -> new Field("name", new BytesRef(), ft));
+  }
 }
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 3ff0664193f..c7381310b37 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java
@@ -880,17 +880,21 @@ public class TestDocValuesIndexing extends LuceneTestCase {
     IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
     IndexWriter w = new IndexWriter(dir, iwc);
     Document doc = new Document();
-    FieldType ft = new FieldType(TextField.TYPE_NOT_STORED);
+    FieldType ft = new FieldType(StringField.TYPE_NOT_STORED);
     ft.setDocValuesType(DocValuesType.SORTED);
     ft.freeze();
-    Field field = new Field("test", "value", ft);
-    field.setTokenStream(
-        new TokenStream() {
+    Field field =
+        new Field("test", new BytesRef("value"), ft) {
           @Override
-          public boolean incrementToken() {
-            throw new RuntimeException("no");
+          public TokenStream tokenStream(Analyzer analyzer, TokenStream reuse) {
+            return new TokenStream() {
+              @Override
+              public boolean incrementToken() throws IOException {
+                throw new RuntimeException();
+              }
+            };
           }
-        });
+        };
     doc.add(field);
     expectThrows(
         RuntimeException.class,
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 e9a2fd9141d..156d4f596f6 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
@@ -1166,41 +1166,44 @@ public class TestIndexWriter extends LuceneTestCase {
     FieldType customType = new FieldType(StoredField.TYPE);
     customType.setTokenized(true);
 
-    Field f = new Field("binary", b, 10, 17, customType);
+    final MockTokenizer field1 = new MockTokenizer(MockTokenizer.WHITESPACE, false);
+    Field f =
+        new Field("binary", b, 10, 17, customType) {
+          @Override
+          public TokenStream tokenStream(Analyzer analyzer, TokenStream reuse) {
+            return field1;
+          }
+        };
     // TODO: this is evil, changing the type after creating the field:
     customType.setIndexOptions(IndexOptions.DOCS);
-    final MockTokenizer doc1field1 = new MockTokenizer(MockTokenizer.WHITESPACE, false);
-    doc1field1.setReader(new StringReader("doc1field1"));
-    f.setTokenStream(doc1field1);
+    field1.setReader(new StringReader("doc1field1"));
 
     FieldType customType2 = new FieldType(TextField.TYPE_STORED);
 
-    Field f2 = newField("string", "value", customType2);
-    final MockTokenizer doc1field2 = new MockTokenizer(MockTokenizer.WHITESPACE, false);
-    doc1field2.setReader(new StringReader("doc1field2"));
-    f2.setTokenStream(doc1field2);
+    final MockTokenizer field2 = new MockTokenizer(MockTokenizer.WHITESPACE, false);
+    Field f2 =
+        new Field("string", "value", customType2) {
+          @Override
+          public TokenStream tokenStream(Analyzer analyzer, TokenStream reuse) {
+            return field2;
+          }
+        };
+
+    field2.setReader(new StringReader("doc1field2"));
     doc.add(f);
     doc.add(f2);
     w.addDocument(doc);
 
     // add 2 docs to test in-memory merging
-    final MockTokenizer doc2field1 = new MockTokenizer(MockTokenizer.WHITESPACE, false);
-    doc2field1.setReader(new StringReader("doc2field1"));
-    f.setTokenStream(doc2field1);
-    final MockTokenizer doc2field2 = new MockTokenizer(MockTokenizer.WHITESPACE, false);
-    doc2field2.setReader(new StringReader("doc2field2"));
-    f2.setTokenStream(doc2field2);
+    field1.setReader(new StringReader("doc2field1"));
+    field2.setReader(new StringReader("doc2field2"));
     w.addDocument(doc);
 
     // force segment flush so we can force a segment merge with doc3 later.
     w.commit();
 
-    final MockTokenizer doc3field1 = new MockTokenizer(MockTokenizer.WHITESPACE, false);
-    doc3field1.setReader(new StringReader("doc3field1"));
-    f.setTokenStream(doc3field1);
-    final MockTokenizer doc3field2 = new MockTokenizer(MockTokenizer.WHITESPACE, false);
-    doc3field2.setReader(new StringReader("doc3field2"));
-    f2.setTokenStream(doc3field2);
+    field1.setReader(new StringReader("doc3field1"));
+    field2.setReader(new StringReader("doc3field2"));
 
     w.addDocument(doc);
     w.commit();
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
index 1b36eeca29b..c11c1be3b8f 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
@@ -1497,13 +1497,13 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
       doc.add(newStringField("id", docCount + "", Field.Store.NO));
       doc.add(newTextField("content", "silly content " + docCount, Field.Store.NO));
       if (docCount == 4) {
-        Field f = newTextField("crash", "", Field.Store.NO);
-        doc.add(f);
         MockTokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
         tokenizer.setReader(new StringReader("crash me on the 4th token"));
         tokenizer.setEnableChecks(
             false); // disable workflow checking as we forcefully close() in exceptional cases.
-        f.setTokenStream(new CrashingFilter("crash", tokenizer));
+        Field f =
+            new Field("crash", new CrashingFilter("crash", tokenizer), TextField.TYPE_NOT_STORED);
+        doc.add(f);
       }
     }
 
@@ -1573,13 +1573,13 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
       doc.add(newStringField("id", docCount + "", Field.Store.NO));
       doc.add(newTextField("content", "silly content " + docCount, Field.Store.NO));
       if (docCount == crashAt) {
-        Field f = newTextField("crash", "", Field.Store.NO);
-        doc.add(f);
         MockTokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
         tokenizer.setReader(new StringReader("crash me on the 4th token"));
         tokenizer.setEnableChecks(
             false); // disable workflow checking as we forcefully close() in exceptional cases.
-        f.setTokenStream(new CrashingFilter("crash", tokenizer));
+        Field f =
+            new Field("crash", new CrashingFilter("crash", tokenizer), TextField.TYPE_NOT_STORED);
+        doc.add(f);
       }
     }
 
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestPayloads.java b/lucene/core/src/test/org/apache/lucene/index/TestPayloads.java
index 4174b77827b..e05f3ae6633 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestPayloads.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestPayloads.java
@@ -568,10 +568,9 @@ public class TestPayloads extends LuceneTestCase {
     iwc.setMergePolicy(newLogMergePolicy());
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
     Document doc = new Document();
-    Field field = new TextField("field", "", Field.Store.NO);
     TokenStream ts = new MockTokenizer(MockTokenizer.WHITESPACE, true);
     ((Tokenizer) ts).setReader(new StringReader("here we go"));
-    field.setTokenStream(ts);
+    Field field = new Field("field", ts, TextField.TYPE_NOT_STORED);
     doc.add(field);
     writer.addDocument(doc);
     Token withPayload = new Token("withPayload", 0, 11);
@@ -601,22 +600,20 @@ public class TestPayloads extends LuceneTestCase {
     Directory dir = newDirectory();
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
     Document doc = new Document();
-    Field field = new TextField("field", "", Field.Store.NO);
     TokenStream ts = new MockTokenizer(MockTokenizer.WHITESPACE, true);
+    Field field = new Field("field", ts, TextField.TYPE_NOT_STORED);
     ((Tokenizer) ts).setReader(new StringReader("here we go"));
     field.setTokenStream(ts);
     doc.add(field);
-    Field field2 = new TextField("field", "", Field.Store.NO);
     Token withPayload = new Token("withPayload", 0, 11);
     withPayload.setPayload(new BytesRef("test"));
     ts = new CannedTokenStream(withPayload);
     assertTrue(ts.hasAttribute(PayloadAttribute.class));
-    field2.setTokenStream(ts);
+    Field field2 = new Field("field", ts, TextField.TYPE_NOT_STORED);
     doc.add(field2);
-    Field field3 = new TextField("field", "", Field.Store.NO);
     ts = new MockTokenizer(MockTokenizer.WHITESPACE, true);
     ((Tokenizer) ts).setReader(new StringReader("nopayload"));
-    field3.setTokenStream(ts);
+    Field field3 = new Field("field", ts, TextField.TYPE_NOT_STORED);
     doc.add(field3);
     writer.addDocument(doc);
     DirectoryReader reader = writer.getReader();
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestPayloadsOnVectors.java b/lucene/core/src/test/org/apache/lucene/index/TestPayloadsOnVectors.java
index 8bcb2bff305..5493b1cd3ee 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestPayloadsOnVectors.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestPayloadsOnVectors.java
@@ -47,10 +47,9 @@ public class TestPayloadsOnVectors extends LuceneTestCase {
     customType.setStoreTermVectorPositions(true);
     customType.setStoreTermVectorPayloads(true);
     customType.setStoreTermVectorOffsets(random().nextBoolean());
-    Field field = new Field("field", "", customType);
     TokenStream ts = new MockTokenizer(MockTokenizer.WHITESPACE, true);
     ((Tokenizer) ts).setReader(new StringReader("here we go"));
-    field.setTokenStream(ts);
+    Field field = new Field("field", ts, customType);
     doc.add(field);
     writer.addDocument(doc);
 
@@ -90,22 +89,19 @@ public class TestPayloadsOnVectors extends LuceneTestCase {
     customType.setStoreTermVectorPositions(true);
     customType.setStoreTermVectorPayloads(true);
     customType.setStoreTermVectorOffsets(random().nextBoolean());
-    Field field = new Field("field", "", customType);
     TokenStream ts = new MockTokenizer(MockTokenizer.WHITESPACE, true);
     ((Tokenizer) ts).setReader(new StringReader("here we go"));
-    field.setTokenStream(ts);
+    Field field = new Field("field", ts, customType);
     doc.add(field);
-    Field field2 = new Field("field", "", customType);
     Token withPayload = new Token("withPayload", 0, 11);
     withPayload.setPayload(new BytesRef("test"));
     ts = new CannedTokenStream(withPayload);
     assertTrue(ts.hasAttribute(PayloadAttribute.class));
-    field2.setTokenStream(ts);
+    Field field2 = new Field("field", ts, customType);
     doc.add(field2);
-    Field field3 = new Field("field", "", customType);
     ts = new MockTokenizer(MockTokenizer.WHITESPACE, true);
     ((Tokenizer) ts).setReader(new StringReader("nopayload"));
-    field3.setTokenStream(ts);
+    Field field3 = new Field("field", ts, customType);
     doc.add(field3);
     writer.addDocument(doc);
     DirectoryReader reader = writer.getReader();
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestTerms.java b/lucene/core/src/test/org/apache/lucene/index/TestTerms.java
index 84067ced62c..9508c383746 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestTerms.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestTerms.java
@@ -51,8 +51,6 @@ public class TestTerms extends LuceneTestCase {
     BytesRef maxTerm = null;
     for (int i = 0; i < numDocs; i++) {
       Document doc = new Document();
-      Field field = new TextField("field", "", Field.Store.NO);
-      doc.add(field);
       // System.out.println("  doc " + i);
       CannedBinaryTokenStream.BinaryToken[] tokens =
           new CannedBinaryTokenStream.BinaryToken[atLeast(10)];
@@ -71,7 +69,9 @@ public class TestTerms extends LuceneTestCase {
         }
         tokens[j] = new CannedBinaryTokenStream.BinaryToken(tokenBytes);
       }
-      field.setTokenStream(new CannedBinaryTokenStream(tokens));
+      Field field =
+          new Field("field", new CannedBinaryTokenStream(tokens), TextField.TYPE_NOT_STORED);
+      doc.add(field);
       w.addDocument(doc);
     }