You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2014/11/17 01:43:47 UTC

svn commit: r1640053 [4/10] - in /lucene/dev/branches/lucene6005/lucene: core/src/java/org/apache/lucene/document/ core/src/java/org/apache/lucene/index/ core/src/test/org/apache/lucene/ core/src/test/org/apache/lucene/codecs/compressing/ core/src/test...

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=1640053&r1=1640052&r2=1640053&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java Mon Nov 17 00:43:44 2014
@@ -48,6 +48,7 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.FieldTypes;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.SortedNumericDocValuesField;
@@ -89,481 +90,423 @@ import org.junit.Test;
 
 public class TestIndexWriter extends LuceneTestCase {
 
-    private static final FieldType storedTextType = new FieldType(TextField.TYPE_NOT_STORED);
-    public void testDocCount() throws IOException {
-        Directory dir = newDirectory();
-
-        IndexWriter writer = null;
-        IndexReader reader = null;
-        int i;
-
-        long savedWriteLockTimeout = IndexWriterConfig.getDefaultWriteLockTimeout();
-        try {
-          IndexWriterConfig.setDefaultWriteLockTimeout(2000);
-          assertEquals(2000, IndexWriterConfig.getDefaultWriteLockTimeout());
-          writer  = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
-        } finally {
-          IndexWriterConfig.setDefaultWriteLockTimeout(savedWriteLockTimeout);
-        }
+  public void testDocCount() throws IOException {
+    Directory dir = newDirectory();
 
-        // add 100 documents
-        for (i = 0; i < 100; i++) {
-            addDocWithIndex(writer,i);
-        }
-        assertEquals(100, writer.maxDoc());
-        writer.close();
+    IndexWriter writer = null;
+    IndexReader reader = null;
+    int i;
 
-        // delete 40 documents
-        writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
-                                        .setMergePolicy(NoMergePolicy.INSTANCE));
-        for (i = 0; i < 40; i++) {
-            writer.deleteDocuments(new Term("id", ""+i));
-        }
-        writer.close();
+    long savedWriteLockTimeout = IndexWriterConfig.getDefaultWriteLockTimeout();
+    try {
+      IndexWriterConfig.setDefaultWriteLockTimeout(2000);
+      assertEquals(2000, IndexWriterConfig.getDefaultWriteLockTimeout());
+      writer  = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
+    } finally {
+      IndexWriterConfig.setDefaultWriteLockTimeout(savedWriteLockTimeout);
+    }
 
-        reader = DirectoryReader.open(dir);
-        assertEquals(60, reader.numDocs());
-        reader.close();
+    // add 100 documents
+    for (i = 0; i < 100; i++) {
+      addDocWithIndex(writer, i);
+    }
+    assertEquals(100, writer.maxDoc());
+    writer.close();
 
-        // merge the index down and check that the new doc count is correct
-        writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
-        assertEquals(60, writer.numDocs());
-        writer.forceMerge(1);
-        assertEquals(60, writer.maxDoc());
-        assertEquals(60, writer.numDocs());
-        writer.close();
+    // delete 40 documents
+    writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
+                             .setMergePolicy(NoMergePolicy.INSTANCE));
+    FieldTypes fieldTypes = writer.getFieldTypes();
+    for (i = 0; i < 40; i++) {
+      writer.deleteDocuments(fieldTypes.newIntTerm("id", i));
+    }
+    writer.close();
 
-        // check that the index reader gives the same numbers.
-        reader = DirectoryReader.open(dir);
-        assertEquals(60, reader.maxDoc());
-        assertEquals(60, reader.numDocs());
-        reader.close();
+    reader = DirectoryReader.open(dir);
+    assertEquals(60, reader.numDocs());
+    reader.close();
 
-        // make sure opening a new index for create over
-        // this existing one works correctly:
-        writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
-                                        .setOpenMode(OpenMode.CREATE));
-        assertEquals(0, writer.maxDoc());
-        assertEquals(0, writer.numDocs());
-        writer.close();
-        dir.close();
-    }
+    // merge the index down and check that the new doc count is correct
+    writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
+    assertEquals(60, writer.numDocs());
+    writer.forceMerge(1);
+    assertEquals(60, writer.maxDoc());
+    assertEquals(60, writer.numDocs());
+    writer.close();
 
-    static void addDoc(IndexWriter writer) throws IOException
-    {
-        Document doc = new Document();
-        doc.add(newTextField("content", "aaa", Field.Store.NO));
-        writer.addDocument(doc);
-    }
+    // check that the index reader gives the same numbers.
+    reader = DirectoryReader.open(dir);
+    assertEquals(60, reader.maxDoc());
+    assertEquals(60, reader.numDocs());
+    reader.close();
 
-    static void addDocWithIndex(IndexWriter writer, int index) throws IOException
-    {
-        Document doc = new Document();
-        doc.add(newField("content", "aaa " + index, storedTextType));
-        doc.add(newField("id", "" + index, storedTextType));
-        writer.addDocument(doc);
-    }
+    // make sure opening a new index for create over
+    // this existing one works correctly:
+    writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
+                             .setOpenMode(OpenMode.CREATE));
+    assertEquals(0, writer.maxDoc());
+    assertEquals(0, writer.numDocs());
+    writer.close();
+    dir.close();
+  }
 
+  static void addDoc(IndexWriter writer) throws IOException {
+    Document2 doc = writer.newDocument();
+    doc.addLargeText("content", "aaa");
+    writer.addDocument(doc);
+  }
 
+  static void addDocWithIndex(IndexWriter writer, int index) throws IOException {
+    Document2 doc = writer.newDocument();
+    doc.addLargeText("content", "aaa " + index);
+    doc.addUniqueInt("id", index);
+    writer.addDocument(doc);
+  }
 
-    // TODO: we have the logic in MDW to do this check, and its better, because it knows about files it tried
-    // to delete but couldn't: we should replace this!!!!
-    public static void assertNoUnreferencedFiles(Directory dir, String message) throws IOException {
-      if (dir instanceof MockDirectoryWrapper) {
-        assertFalse("test is broken: should disable virus scanner", ((MockDirectoryWrapper)dir).getEnableVirusScanner());
-      }
-      String[] startFiles = dir.listAll();
-      new IndexWriter(dir, new IndexWriterConfig(new MockAnalyzer(random()))).rollback();
-      String[] endFiles = dir.listAll();
+  // TODO: we have the logic in MDW to do this check, and its better, because it knows about files it tried
+  // to delete but couldn't: we should replace this!!!!
+  public static void assertNoUnreferencedFiles(Directory dir, String message) throws IOException {
+    if (dir instanceof MockDirectoryWrapper) {
+      assertFalse("test is broken: should disable virus scanner", ((MockDirectoryWrapper)dir).getEnableVirusScanner());
+    }
+    String[] startFiles = dir.listAll();
+    new IndexWriter(dir, new IndexWriterConfig(new MockAnalyzer(random()))).rollback();
+    String[] endFiles = dir.listAll();
 
-      Arrays.sort(startFiles);
-      Arrays.sort(endFiles);
+    Arrays.sort(startFiles);
+    Arrays.sort(endFiles);
 
-      if (!Arrays.equals(startFiles, endFiles)) {
-        fail(message + ": before delete:\n    " + arrayToString(startFiles) + "\n  after delete:\n    " + arrayToString(endFiles));
-      }
+    if (!Arrays.equals(startFiles, endFiles)) {
+      fail(message + ": before delete:\n    " + arrayToString(startFiles) + "\n  after delete:\n    " + arrayToString(endFiles));
     }
+  }
 
-    static String arrayToString(String[] l) {
-      String s = "";
-      for(int i=0;i<l.length;i++) {
-        if (i > 0) {
-          s += "\n    ";
-        }
-        s += l[i];
+  static String arrayToString(String[] l) {
+    String s = "";
+    for(int i=0;i<l.length;i++) {
+      if (i > 0) {
+        s += "\n    ";
       }
-      return s;
+      s += l[i];
     }
+    return s;
+  }
 
-    // Make sure we can open an index for create even when a
-    // reader holds it open (this fails pre lock-less
-    // commits on windows):
-    public void testCreateWithReader() throws IOException {
-      Directory dir = newDirectory();
+  // Make sure we can open an index for create even when a
+  // reader holds it open (this fails pre lock-less
+  // commits on windows):
+  public void testCreateWithReader() throws IOException {
+    Directory dir = newDirectory();
 
-      // add one document & close writer
-      IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
-      addDoc(writer);
-      writer.close();
+    // add one document & close writer
+    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
+    addDoc(writer);
+    writer.close();
 
-      // now open reader:
-      IndexReader reader = DirectoryReader.open(dir);
-      assertEquals("should be one document", reader.numDocs(), 1);
-
-      // now open index for create:
-      writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
-                                      .setOpenMode(OpenMode.CREATE));
-      assertEquals("should be zero documents", writer.maxDoc(), 0);
-      addDoc(writer);
-      writer.close();
+    // now open reader:
+    IndexReader reader = DirectoryReader.open(dir);
+    assertEquals("should be one document", reader.numDocs(), 1);
 
-      assertEquals("should be one document", reader.numDocs(), 1);
-      IndexReader reader2 = DirectoryReader.open(dir);
-      assertEquals("should be one document", reader2.numDocs(), 1);
-      reader.close();
-      reader2.close();
+    // now open index for create:
+    writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
+                             .setOpenMode(OpenMode.CREATE));
+    assertEquals("should be zero documents", writer.maxDoc(), 0);
+    addDoc(writer);
+    writer.close();
 
-      dir.close();
-    }
+    assertEquals("should be one document", reader.numDocs(), 1);
+    IndexReader reader2 = DirectoryReader.open(dir);
+    assertEquals("should be one document", reader2.numDocs(), 1);
+    reader.close();
+    reader2.close();
 
-    public void testChangesAfterClose() throws IOException {
-        Directory dir = newDirectory();
+    dir.close();
+  }
 
-        IndexWriter writer = null;
+  public void testChangesAfterClose() throws IOException {
+    Directory dir = newDirectory();
 
-        writer  = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
-        addDoc(writer);
+    IndexWriter writer = null;
 
-        // close
-        writer.close();
-        try {
-          addDoc(writer);
-          fail("did not hit AlreadyClosedException");
-        } catch (AlreadyClosedException e) {
-          // expected
-        }
-        dir.close();
+    writer  = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
+    addDoc(writer);
+
+    // close
+    writer.close();
+    try {
+      addDoc(writer);
+      fail("did not hit AlreadyClosedException");
+    } catch (AlreadyClosedException e) {
+      // expected
     }
+    dir.close();
+  }
 
 
 
-    public void testIndexNoDocuments() throws IOException {
-      Directory dir = newDirectory();
-      IndexWriter writer  = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
-      writer.commit();
-      writer.close();
-
-      IndexReader reader = DirectoryReader.open(dir);
-      assertEquals(0, reader.maxDoc());
-      assertEquals(0, reader.numDocs());
-      reader.close();
-
-      writer  = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
-                                       .setOpenMode(OpenMode.APPEND));
-      writer.commit();
-      writer.close();
-
-      reader = DirectoryReader.open(dir);
-      assertEquals(0, reader.maxDoc());
-      assertEquals(0, reader.numDocs());
-      reader.close();
-      dir.close();
-    }
+  public void testIndexNoDocuments() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriter writer  = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
+    writer.commit();
+    writer.close();
 
-    public void testSmallRAMBuffer() throws IOException {
-      Directory dir = newDirectory();
-      IndexWriter writer  = new IndexWriter(
-          dir,
-          newIndexWriterConfig(new MockAnalyzer(random()))
-              .setRAMBufferSizeMB(0.000001)
-              .setMergePolicy(newLogMergePolicy(10))
-      );
-      int lastNumSegments = getSegmentCount(dir);
-      for(int j=0;j<9;j++) {
-        Document doc = new Document();
-        doc.add(newField("field", "aaa" + j, storedTextType));
-        writer.addDocument(doc);
-        // Verify that with a tiny RAM buffer we see new
-        // segment after every doc
-        int numSegments = getSegmentCount(dir);
-        assertTrue(numSegments > lastNumSegments);
-        lastNumSegments = numSegments;
-      }
-      writer.close();
-      dir.close();
+    IndexReader reader = DirectoryReader.open(dir);
+    assertEquals(0, reader.maxDoc());
+    assertEquals(0, reader.numDocs());
+    reader.close();
+
+    writer  = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
+                              .setOpenMode(OpenMode.APPEND));
+    writer.commit();
+    writer.close();
+
+    reader = DirectoryReader.open(dir);
+    assertEquals(0, reader.maxDoc());
+    assertEquals(0, reader.numDocs());
+    reader.close();
+    dir.close();
+  }
+
+  public void testSmallRAMBuffer() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriter writer  = new IndexWriter(
+                                          dir,
+                                          newIndexWriterConfig(new MockAnalyzer(random()))
+                                          .setRAMBufferSizeMB(0.000001)
+                                          .setMergePolicy(newLogMergePolicy(10))
+                                          );
+    int lastNumSegments = getSegmentCount(dir);
+    for(int j=0;j<9;j++) {
+      Document2 doc = writer.newDocument();
+      doc.addLargeText("field", "aaa" + j);
+      writer.addDocument(doc);
+      // Verify that with a tiny RAM buffer we see new
+      // segment after every doc
+      int numSegments = getSegmentCount(dir);
+      assertTrue(numSegments > lastNumSegments);
+      lastNumSegments = numSegments;
     }
+    writer.close();
+    dir.close();
+  }
 
-    /** Returns how many unique segment names are in the directory. */
-    private static int getSegmentCount(Directory dir) throws IOException {
-      Set<String> segments = new HashSet<>();
-      for(String file : dir.listAll()) {
-        segments.add(IndexFileNames.parseSegmentName(file));
-      }
-
-      return segments.size();
-    }
-
-    // Make sure it's OK to change RAM buffer size and
-    // maxBufferedDocs in a write session
-    public void testChangingRAMBuffer() throws IOException {
-      Directory dir = newDirectory();      
-      IndexWriter writer  = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
-      writer.getConfig().setMaxBufferedDocs(10);
-      writer.getConfig().setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
-
-      int lastFlushCount = -1;
-      for(int j=1;j<52;j++) {
-        Document doc = new Document();
-        doc.add(new Field("field", "aaa" + j, storedTextType));
-        writer.addDocument(doc);
-        TestUtil.syncConcurrentMerges(writer);
-        int flushCount = writer.getFlushCount();
-        if (j == 1)
-          lastFlushCount = flushCount;
-        else if (j < 10)
-          // No new files should be created
-          assertEquals(flushCount, lastFlushCount);
-        else if (10 == j) {
-          assertTrue(flushCount > lastFlushCount);
-          lastFlushCount = flushCount;
-          writer.getConfig().setRAMBufferSizeMB(0.000001);
-          writer.getConfig().setMaxBufferedDocs(IndexWriterConfig.DISABLE_AUTO_FLUSH);
-        } else if (j < 20) {
-          assertTrue(flushCount > lastFlushCount);
-          lastFlushCount = flushCount;
-        } else if (20 == j) {
-          writer.getConfig().setRAMBufferSizeMB(16);
-          writer.getConfig().setMaxBufferedDocs(IndexWriterConfig.DISABLE_AUTO_FLUSH);
-          lastFlushCount = flushCount;
-        } else if (j < 30) {
-          assertEquals(flushCount, lastFlushCount);
-        } else if (30 == j) {
-          writer.getConfig().setRAMBufferSizeMB(0.000001);
-          writer.getConfig().setMaxBufferedDocs(IndexWriterConfig.DISABLE_AUTO_FLUSH);
-        } else if (j < 40) {
-          assertTrue(flushCount> lastFlushCount);
-          lastFlushCount = flushCount;
-        } else if (40 == j) {
-          writer.getConfig().setMaxBufferedDocs(10);
-          writer.getConfig().setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
-          lastFlushCount = flushCount;
-        } else if (j < 50) {
-          assertEquals(flushCount, lastFlushCount);
-          writer.getConfig().setMaxBufferedDocs(10);
-          writer.getConfig().setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
-        } else if (50 == j) {
-          assertTrue(flushCount > lastFlushCount);
-        }
+  /** Returns how many unique segment names are in the directory. */
+  private static int getSegmentCount(Directory dir) throws IOException {
+    Set<String> segments = new HashSet<>();
+    for(String file : dir.listAll()) {
+      segments.add(IndexFileNames.parseSegmentName(file));
+    }
+
+    return segments.size();
+  }
+
+  // Make sure it's OK to change RAM buffer size and
+  // maxBufferedDocs in a write session
+  public void testChangingRAMBuffer() throws IOException {
+    Directory dir = newDirectory();      
+    IndexWriter writer  = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
+    writer.getConfig().setMaxBufferedDocs(10);
+    writer.getConfig().setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
+
+    int lastFlushCount = -1;
+    for(int j=1;j<52;j++) {
+      Document2 doc = writer.newDocument();
+      doc.addLargeText("field", "aaa" + j);
+      writer.addDocument(doc);
+      TestUtil.syncConcurrentMerges(writer);
+      int flushCount = writer.getFlushCount();
+      if (j == 1)
+        lastFlushCount = flushCount;
+      else if (j < 10)
+        // No new files should be created
+        assertEquals(flushCount, lastFlushCount);
+      else if (10 == j) {
+        assertTrue(flushCount > lastFlushCount);
+        lastFlushCount = flushCount;
+        writer.getConfig().setRAMBufferSizeMB(0.000001);
+        writer.getConfig().setMaxBufferedDocs(IndexWriterConfig.DISABLE_AUTO_FLUSH);
+      } else if (j < 20) {
+        assertTrue(flushCount > lastFlushCount);
+        lastFlushCount = flushCount;
+      } else if (20 == j) {
+        writer.getConfig().setRAMBufferSizeMB(16);
+        writer.getConfig().setMaxBufferedDocs(IndexWriterConfig.DISABLE_AUTO_FLUSH);
+        lastFlushCount = flushCount;
+      } else if (j < 30) {
+        assertEquals(flushCount, lastFlushCount);
+      } else if (30 == j) {
+        writer.getConfig().setRAMBufferSizeMB(0.000001);
+        writer.getConfig().setMaxBufferedDocs(IndexWriterConfig.DISABLE_AUTO_FLUSH);
+      } else if (j < 40) {
+        assertTrue(flushCount> lastFlushCount);
+        lastFlushCount = flushCount;
+      } else if (40 == j) {
+        writer.getConfig().setMaxBufferedDocs(10);
+        writer.getConfig().setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
+        lastFlushCount = flushCount;
+      } else if (j < 50) {
+        assertEquals(flushCount, lastFlushCount);
+        writer.getConfig().setMaxBufferedDocs(10);
+        writer.getConfig().setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
+      } else if (50 == j) {
+        assertTrue(flushCount > lastFlushCount);
       }
-      writer.close();
-      dir.close();
     }
+    writer.close();
+    dir.close();
+  }
 
-    public void testChangingRAMBuffer2() throws IOException {
-      Directory dir = newDirectory();      
-      IndexWriter writer  = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
-      writer.getConfig().setMaxBufferedDocs(10);
-      writer.getConfig().setMaxBufferedDeleteTerms(10);
-      writer.getConfig().setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
-
-      for(int j=1;j<52;j++) {
-        Document doc = new Document();
-        doc.add(new Field("field", "aaa" + j, storedTextType));
-        writer.addDocument(doc);
-      }
+  public void testChangingRAMBuffer2() throws IOException {
+    Directory dir = newDirectory();      
+    IndexWriter writer  = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
+    writer.getConfig().setMaxBufferedDocs(10);
+    writer.getConfig().setMaxBufferedDeleteTerms(10);
+    writer.getConfig().setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
+
+    for(int j=1;j<52;j++) {
+      Document2 doc = writer.newDocument();
+      doc.addLargeText("field", "aaa" + j);
+      writer.addDocument(doc);
+    }
       
-      int lastFlushCount = -1;
-      for(int j=1;j<52;j++) {
-        writer.deleteDocuments(new Term("field", "aaa" + j));
-        TestUtil.syncConcurrentMerges(writer);
-        int flushCount = writer.getFlushCount();
+    int lastFlushCount = -1;
+    for(int j=1;j<52;j++) {
+      writer.deleteDocuments(new Term("field", "aaa" + j));
+      TestUtil.syncConcurrentMerges(writer);
+      int flushCount = writer.getFlushCount();
        
-        if (j == 1)
-          lastFlushCount = flushCount;
-        else if (j < 10) {
-          // No new files should be created
-          assertEquals(flushCount, lastFlushCount);
-        } else if (10 == j) {
-          assertTrue("" + j, flushCount > lastFlushCount);
-          lastFlushCount = flushCount;
-          writer.getConfig().setRAMBufferSizeMB(0.000001);
-          writer.getConfig().setMaxBufferedDeleteTerms(1);
-        } else if (j < 20) {
-          assertTrue(flushCount > lastFlushCount);
-          lastFlushCount = flushCount;
-        } else if (20 == j) {
-          writer.getConfig().setRAMBufferSizeMB(16);
-          writer.getConfig().setMaxBufferedDeleteTerms(IndexWriterConfig.DISABLE_AUTO_FLUSH);
-          lastFlushCount = flushCount;
-        } else if (j < 30) {
-          assertEquals(flushCount, lastFlushCount);
-        } else if (30 == j) {
-          writer.getConfig().setRAMBufferSizeMB(0.000001);
-          writer.getConfig().setMaxBufferedDeleteTerms(IndexWriterConfig.DISABLE_AUTO_FLUSH);
-          writer.getConfig().setMaxBufferedDeleteTerms(1);
-        } else if (j < 40) {
-          assertTrue(flushCount> lastFlushCount);
-          lastFlushCount = flushCount;
-        } else if (40 == j) {
-          writer.getConfig().setMaxBufferedDeleteTerms(10);
-          writer.getConfig().setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
-          lastFlushCount = flushCount;
-        } else if (j < 50) {
-          assertEquals(flushCount, lastFlushCount);
-          writer.getConfig().setMaxBufferedDeleteTerms(10);
-          writer.getConfig().setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
-        } else if (50 == j) {
-          assertTrue(flushCount > lastFlushCount);
-        }
+      if (j == 1)
+        lastFlushCount = flushCount;
+      else if (j < 10) {
+        // No new files should be created
+        assertEquals(flushCount, lastFlushCount);
+      } else if (10 == j) {
+        assertTrue("" + j, flushCount > lastFlushCount);
+        lastFlushCount = flushCount;
+        writer.getConfig().setRAMBufferSizeMB(0.000001);
+        writer.getConfig().setMaxBufferedDeleteTerms(1);
+      } else if (j < 20) {
+        assertTrue(flushCount > lastFlushCount);
+        lastFlushCount = flushCount;
+      } else if (20 == j) {
+        writer.getConfig().setRAMBufferSizeMB(16);
+        writer.getConfig().setMaxBufferedDeleteTerms(IndexWriterConfig.DISABLE_AUTO_FLUSH);
+        lastFlushCount = flushCount;
+      } else if (j < 30) {
+        assertEquals(flushCount, lastFlushCount);
+      } else if (30 == j) {
+        writer.getConfig().setRAMBufferSizeMB(0.000001);
+        writer.getConfig().setMaxBufferedDeleteTerms(IndexWriterConfig.DISABLE_AUTO_FLUSH);
+        writer.getConfig().setMaxBufferedDeleteTerms(1);
+      } else if (j < 40) {
+        assertTrue(flushCount> lastFlushCount);
+        lastFlushCount = flushCount;
+      } else if (40 == j) {
+        writer.getConfig().setMaxBufferedDeleteTerms(10);
+        writer.getConfig().setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
+        lastFlushCount = flushCount;
+      } else if (j < 50) {
+        assertEquals(flushCount, lastFlushCount);
+        writer.getConfig().setMaxBufferedDeleteTerms(10);
+        writer.getConfig().setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
+      } else if (50 == j) {
+        assertTrue(flushCount > lastFlushCount);
       }
-      writer.close();
-      dir.close();
     }
+    writer.close();
+    dir.close();
+  }
 
-    public void testEnablingNorms() throws IOException {
-      Directory dir = newDirectory();
-      IndexWriter writer  = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
-                                                   .setMaxBufferedDocs(10));
-      // Enable norms for only 1 doc, pre flush
-      FieldType customType = new FieldType(TextField.TYPE_STORED);
-      customType.setOmitNorms(true);
-      for(int j=0;j<10;j++) {
-        Document doc = new Document();
-        Field f = null;
-        if (j != 8) {
-          f = newField("field", "aaa", customType);
-        }
-        else {
-          f = newField("field", "aaa", storedTextType);
-        }
-        doc.add(f);
-        writer.addDocument(doc);
-      }
-      writer.close();
-
-      Term searchTerm = new Term("field", "aaa");
-
-      IndexReader reader = DirectoryReader.open(dir);
-      IndexSearcher searcher = newSearcher(reader);
-      ScoreDoc[] hits = searcher.search(new TermQuery(searchTerm), null, 1000).scoreDocs;
-      assertEquals(10, hits.length);
-      reader.close();
-
-      writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
-                                      .setOpenMode(OpenMode.CREATE).setMaxBufferedDocs(10));
-      // Enable norms for only 1 doc, post flush
-      for(int j=0;j<27;j++) {
-        Document doc = new Document();
-        Field f = null;
-        if (j != 26) {
-          f = newField("field", "aaa", customType);
-        }
-        else {
-          f = newField("field", "aaa", storedTextType);
-        }
-        doc.add(f);
-        writer.addDocument(doc);
-      }
-      writer.close();
-      reader = DirectoryReader.open(dir);
-      searcher = newSearcher(reader);
-      hits = searcher.search(new TermQuery(searchTerm), null, 1000).scoreDocs;
-      assertEquals(27, hits.length);
-      reader.close();
+  public void testHighFreqTerm() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
+                                         .setRAMBufferSizeMB(0.01));
+    // Massive doc that has 128 K a's
+    StringBuilder b = new StringBuilder(1024*1024);
+    for(int i=0;i<4096;i++) {
+      b.append(" a a a a a a a a");
+      b.append(" a a a a a a a a");
+      b.append(" a a a a a a a a");
+      b.append(" a a a a a a a a");
+    }
+    FieldTypes fieldTypes = writer.getFieldTypes();
+    fieldTypes.enableTermVectors("field");
+    fieldTypes.enableTermVectorPositions("field");
+    fieldTypes.enableTermVectorOffsets("field");
 
-      reader = DirectoryReader.open(dir);
-      reader.close();
+    Document2 doc = writer.newDocument();
+    doc.addLargeText("field", b.toString());
+    writer.addDocument(doc);
+    writer.close();
 
-      dir.close();
-    }
+    IndexReader reader = DirectoryReader.open(dir);
+    assertEquals(1, reader.maxDoc());
+    assertEquals(1, reader.numDocs());
+    Term t = new Term("field", "a");
+    assertEquals(1, reader.docFreq(t));
+    DocsEnum td = TestUtil.docs(random(), reader,
+                                "field",
+                                new BytesRef("a"),
+                                MultiFields.getLiveDocs(reader),
+                                null,
+                                DocsEnum.FLAG_FREQS);
+    td.nextDoc();
+    assertEquals(128*1024, td.freq());
+    reader.close();
+    dir.close();
+  }
 
-    public void testHighFreqTerm() throws IOException {
-      Directory dir = newDirectory();
-      IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
-                                                  .setRAMBufferSizeMB(0.01));
-      // Massive doc that has 128 K a's
-      StringBuilder b = new StringBuilder(1024*1024);
-      for(int i=0;i<4096;i++) {
-        b.append(" a a a a a a a a");
-        b.append(" a a a a a a a a");
-        b.append(" a a a a a a a a");
-        b.append(" a a a a a a a a");
-      }
-      Document doc = new Document();
-      FieldType customType = new FieldType(TextField.TYPE_STORED);
-      customType.setStoreTermVectors(true);
-      customType.setStoreTermVectorPositions(true);
-      customType.setStoreTermVectorOffsets(true);
-      doc.add(newField("field", b.toString(), customType));
+  public void testFlushWithNoMerging() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriter writer = new IndexWriter(
+                                         dir,
+                                         newIndexWriterConfig(new MockAnalyzer(random()))
+                                         .setMaxBufferedDocs(2)
+                                         .setMergePolicy(newLogMergePolicy(10))
+                                         );
+    FieldTypes fieldTypes = writer.getFieldTypes();
+    fieldTypes.enableTermVectors("field");
+    fieldTypes.enableTermVectorPositions("field");
+    fieldTypes.enableTermVectorOffsets("field");
+
+    Document2 doc = writer.newDocument();
+    doc.addLargeText("field", "aaa");
+    for(int i=0;i<19;i++) {
       writer.addDocument(doc);
-      writer.close();
-
-      IndexReader reader = DirectoryReader.open(dir);
-      assertEquals(1, reader.maxDoc());
-      assertEquals(1, reader.numDocs());
-      Term t = new Term("field", "a");
-      assertEquals(1, reader.docFreq(t));
-      DocsEnum td = TestUtil.docs(random(), reader,
-          "field",
-          new BytesRef("a"),
-          MultiFields.getLiveDocs(reader),
-          null,
-          DocsEnum.FLAG_FREQS);
-      td.nextDoc();
-      assertEquals(128*1024, td.freq());
-      reader.close();
-      dir.close();
     }
+    writer.flush(false, true);
+    writer.close();
+    SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
+    // Since we flushed w/o allowing merging we should now
+    // have 10 segments
+    assertEquals(10, sis.size());
+    dir.close();
+  }
 
-    public void testFlushWithNoMerging() throws IOException {
-      Directory dir = newDirectory();
-      IndexWriter writer = new IndexWriter(
-          dir,
-          newIndexWriterConfig(new MockAnalyzer(random()))
-              .setMaxBufferedDocs(2)
-              .setMergePolicy(newLogMergePolicy(10))
-      );
-      Document doc = new Document();
-      FieldType customType = new FieldType(TextField.TYPE_STORED);
-      customType.setStoreTermVectors(true);
-      customType.setStoreTermVectorPositions(true);
-      customType.setStoreTermVectorOffsets(true);
-      doc.add(newField("field", "aaa", customType));
-      for(int i=0;i<19;i++)
-        writer.addDocument(doc);
-      writer.flush(false, true);
-      writer.close();
-      SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
-      // Since we flushed w/o allowing merging we should now
-      // have 10 segments
-      assertEquals(10, sis.size());
-      dir.close();
-    }
+  // Make sure we can flush segment w/ norms, then add
+  // empty doc (no norms) and flush
+  public void testEmptyDocAfterFlushingRealDoc() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriter writer  = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
+    FieldTypes fieldTypes = writer.getFieldTypes();
+    fieldTypes.enableTermVectors("field");
+    fieldTypes.enableTermVectorPositions("field");
+    fieldTypes.enableTermVectorOffsets("field");
 
-    // Make sure we can flush segment w/ norms, then add
-    // empty doc (no norms) and flush
-    public void testEmptyDocAfterFlushingRealDoc() throws IOException {
-      Directory dir = newDirectory();
-      IndexWriter writer  = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
-      Document doc = new Document();
-      FieldType customType = new FieldType(TextField.TYPE_STORED);
-      customType.setStoreTermVectors(true);
-      customType.setStoreTermVectorPositions(true);
-      customType.setStoreTermVectorOffsets(true);
-      doc.add(newField("field", "aaa", customType));
-      writer.addDocument(doc);
-      writer.commit();
-      if (VERBOSE) {
-        System.out.println("\nTEST: now add empty doc");
-      }
-      writer.addDocument(new Document());
-      writer.close();
-      IndexReader reader = DirectoryReader.open(dir);
-      assertEquals(2, reader.numDocs());
-      reader.close();
-      dir.close();
+    Document2 doc = writer.newDocument();
+    doc.addLargeText("field", "aaa");
+    writer.addDocument(doc);
+    writer.commit();
+    if (VERBOSE) {
+      System.out.println("\nTEST: now add empty doc");
     }
-
+    writer.addDocument(writer.newDocument());
+    writer.close();
+    IndexReader reader = DirectoryReader.open(dir);
+    assertEquals(2, reader.numDocs());
+    reader.close();
+    dir.close();
+  }
 
 
   /**
@@ -575,11 +518,12 @@ public class TestIndexWriter extends Luc
     Directory dir = newDirectory();
     IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
 
-    Document document = new Document();
-    FieldType customType = new FieldType(TextField.TYPE_NOT_STORED);
-    customType.setStoreTermVectors(true);
-    document.add(newField("tvtest", "", customType));
-    iw.addDocument(document);
+    FieldTypes fieldTypes = iw.getFieldTypes();
+    fieldTypes.enableTermVectors("tvtest");
+
+    Document2 doc = iw.newDocument();
+    doc.addLargeText("tvtest", "");
+    iw.addDocument(doc);
     iw.close();
     dir.close();
   }
@@ -594,10 +538,11 @@ public class TestIndexWriter extends Luc
                                  .setMergePolicy(newLogMergePolicy());
       ((LogMergePolicy) conf.getMergePolicy()).setMergeFactor(2);
       IndexWriter iw = new IndexWriter(dir, conf);
-      Document document = new Document();
-      FieldType customType = new FieldType(TextField.TYPE_NOT_STORED);
-      customType.setStoreTermVectors(true);
-      document.add(newField("tvtest", "a b c", customType));
+
+      FieldTypes fieldTypes = iw.getFieldTypes();
+      fieldTypes.enableTermVectors("tvtest");
+      Document2 document = iw.newDocument();
+      document.addLargeText("tvtest", "a b c");
       Thread.currentThread().setPriority(Thread.MAX_PRIORITY);
       for(int i=0;i<4;i++)
         iw.addDocument(document);
@@ -608,65 +553,19 @@ public class TestIndexWriter extends Luc
     }
   }
 
-  public void testVariableSchema() throws Exception {
-    Directory dir = newDirectory();
-    for(int i=0;i<20;i++) {
-      if (VERBOSE) {
-        System.out.println("TEST: iter=" + i);
-      }
-      IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
-                                                  .setMaxBufferedDocs(2)
-                                                  .setMergePolicy(newLogMergePolicy()));
-      //LogMergePolicy lmp = (LogMergePolicy) writer.getConfig().getMergePolicy();
-      //lmp.setMergeFactor(2);
-      //lmp.setNoCFSRatio(0.0);
-      Document doc = new Document();
-      String contents = "aa bb cc dd ee ff gg hh ii jj kk";
-
-      FieldType customType = new FieldType(TextField.TYPE_STORED);
-      FieldType type = null;
-      if (i == 7) {
-        // Add empty docs here
-        doc.add(newTextField("content3", "", Field.Store.NO));
-      } else {
-        if (i%2 == 0) {
-          doc.add(newField("content4", contents, customType));
-          type = customType;
-        } else
-          type = TextField.TYPE_NOT_STORED; 
-        doc.add(newTextField("content1", contents, Field.Store.NO));
-        doc.add(newField("content3", "", customType));
-        doc.add(newField("content5", "", type));
-      }
-
-      for(int j=0;j<4;j++)
-        writer.addDocument(doc);
-
-      writer.close();
-
-      if (0 == i % 4) {
-        writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
-        //LogMergePolicy lmp2 = (LogMergePolicy) writer.getConfig().getMergePolicy();
-        //lmp2.setNoCFSRatio(0.0);
-        writer.forceMerge(1);
-        writer.close();
-      }
-    }
-    dir.close();
-  }
-
   // LUCENE-1084: test unlimited field length
   public void testUnlimitedMaxFieldLength() throws IOException {
     Directory dir = newDirectory();
 
     IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
 
-    Document doc = new Document();
+    Document2 doc = writer.newDocument();
     StringBuilder b = new StringBuilder();
-    for(int i=0;i<10000;i++)
+    for(int i=0;i<10000;i++) {
       b.append(" a");
+    }
     b.append(" x");
-    doc.add(newTextField("field", b.toString(), Field.Store.NO));
+    doc.addLargeText("field", b.toString());
     writer.addDocument(doc);
     writer.close();
 
@@ -677,14 +576,12 @@ public class TestIndexWriter extends Luc
     dir.close();
   }
 
-
-
   // LUCENE-1179
   public void testEmptyFieldName() throws IOException {
     Directory dir = newDirectory();
     IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
-    Document doc = new Document();
-    doc.add(newTextField("", "a b c", Field.Store.NO));
+    Document2 doc = writer.newDocument();
+    doc.addLargeText("", "a b c");
     writer.addDocument(doc);
     writer.close();
     dir.close();
@@ -693,8 +590,8 @@ public class TestIndexWriter extends Luc
   public void testEmptyFieldNameTerms() throws IOException {
     Directory dir = newDirectory();
     IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
-    Document doc = new Document();
-    doc.add(newTextField("", "a b c", Field.Store.NO));
+    Document2 doc = writer.newDocument();
+    doc.addLargeText("", "a b c");
     writer.addDocument(doc);  
     writer.close();
     DirectoryReader reader = DirectoryReader.open(dir);
@@ -711,11 +608,13 @@ public class TestIndexWriter extends Luc
   public void testEmptyFieldNameWithEmptyTerm() throws IOException {
     Directory dir = newDirectory();
     IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
-    Document doc = new Document();
-    doc.add(newStringField("", "", Field.Store.NO));
-    doc.add(newStringField("", "a", Field.Store.NO));
-    doc.add(newStringField("", "b", Field.Store.NO));
-    doc.add(newStringField("", "c", Field.Store.NO));
+    FieldTypes fieldTypes = writer.getFieldTypes();
+    fieldTypes.setMultiValued("");
+    Document2 doc = writer.newDocument();
+    doc.addAtom("", "");
+    doc.addAtom("", "a");
+    doc.addAtom("", "b");
+    doc.addAtom("", "c");
     writer.addDocument(doc);  
     writer.close();
     DirectoryReader reader = DirectoryReader.open(dir);
@@ -730,8 +629,6 @@ public class TestIndexWriter extends Luc
     dir.close();
   }
 
-
-
   private static final class MockIndexWriter extends IndexWriter {
 
     public MockIndexWriter(Directory dir, IndexWriterConfig conf) throws IOException {
@@ -757,9 +654,8 @@ public class TestIndexWriter extends Luc
   public void testDoBeforeAfterFlush() throws IOException {
     Directory dir = newDirectory();
     MockIndexWriter w = new MockIndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
-    Document doc = new Document();
-    FieldType customType = new FieldType(TextField.TYPE_STORED);
-    doc.add(newField("field", "a field", customType));
+    Document2 doc = w.newDocument();
+    doc.addLargeText("field", "a field");
     w.addDocument(doc);
     w.commit();
     assertTrue(w.beforeWasCalled);
@@ -801,8 +697,8 @@ public class TestIndexWriter extends Luc
 
     Directory dir = newDirectory();
     IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
-    Document doc = new Document();
-    doc.add(new TextField("field", tokens));
+    Document2 doc = w.newDocument();
+    doc.addLargeText("field", tokens);
     try {
       w.addDocument(doc);
       fail("did not hit expected exception");
@@ -819,14 +715,12 @@ public class TestIndexWriter extends Luc
     MockAnalyzer analyzer = new MockAnalyzer(random());
     analyzer.setPositionIncrementGap( 100 );
     IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(analyzer));
-    Document doc = new Document();
-    FieldType customType = new FieldType(TextField.TYPE_NOT_STORED);
-    customType.setStoreTermVectors(true);
-    customType.setStoreTermVectorPositions(true);
-    Field f = newField("field", "", customType);
-    Field f2 = newField("field", "crunch man", customType);
-    doc.add(f);
-    doc.add(f2);
+    FieldTypes fieldTypes = w.getFieldTypes();
+    fieldTypes.enableTermVectors("field");
+    fieldTypes.enableTermVectorPositions("field");
+    Document2 doc = w.newDocument();
+    doc.addLargeText("field", "");
+    doc.addLargeText("field", "crunch man");
     w.addDocument(doc);
     w.close();
 
@@ -856,14 +750,13 @@ public class TestIndexWriter extends Luc
     Directory dir = newDirectory();
     IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
                                                 .setMaxBufferedDocs(2));
-    Document doc = new Document();
+    FieldTypes fieldTypes = writer.getFieldTypes();
+    fieldTypes.enableTermVectors("content");
+    fieldTypes.enableTermVectorPositions("content");
+    fieldTypes.enableTermVectorOffsets("content");
 
-    FieldType customType = new FieldType(TextField.TYPE_STORED);
-    customType.setStoreTermVectors(true);
-    customType.setStoreTermVectorPositions(true);
-    customType.setStoreTermVectorOffsets(true);
-    
-    doc.add(newField("content", "aaa bbb ccc ddd eee fff ggg hhh iii", customType));
+    Document2 doc = writer.newDocument();
+    doc.addLargeText("content", "aaa bbb ccc ddd eee fff ggg hhh iii");
     writer.addDocument(doc);
     writer.addDocument(doc);
     writer.addDocument(doc);
@@ -872,7 +765,16 @@ public class TestIndexWriter extends Luc
 
     Directory dir2 = newDirectory();
     IndexWriter writer2 = new IndexWriter(dir2, newIndexWriterConfig(new MockAnalyzer(random())));
+
+    fieldTypes = writer2.getFieldTypes();
+    fieldTypes.enableTermVectors("content");
+    fieldTypes.enableTermVectorPositions("content");
+    fieldTypes.enableTermVectorOffsets("content");
+
+    doc = writer2.newDocument();
+    doc.addLargeText("content", "aaa bbb ccc ddd eee fff ggg hhh iii");
     writer2.addDocument(doc);
+    writer2.commit();
     writer2.close();
 
     IndexReader r1 = DirectoryReader.open(dir2);
@@ -908,29 +810,36 @@ public class TestIndexWriter extends Luc
       adder = new MockDirectoryWrapper(random, new RAMDirectory());
       IndexWriterConfig conf = newIndexWriterConfig(random, new MockAnalyzer(random));
       IndexWriter w = new IndexWriter(adder, conf);
-      Document doc = new Document();
-      doc.add(newStringField(random, "id", "500", Field.Store.NO));
-      doc.add(newField(random, "field", "some prepackaged text contents", storedTextType));
-      doc.add(new BinaryDocValuesField("binarydv", new BytesRef("500")));
-      doc.add(new NumericDocValuesField("numericdv", 500));
-      doc.add(new SortedDocValuesField("sorteddv", new BytesRef("500")));
-      doc.add(new SortedSetDocValuesField("sortedsetdv", new BytesRef("one")));
-      doc.add(new SortedSetDocValuesField("sortedsetdv", new BytesRef("two")));
-      doc.add(new SortedNumericDocValuesField("sortednumericdv", 4));
-      doc.add(new SortedNumericDocValuesField("sortednumericdv", 3));
+      FieldTypes fieldTypes = w.getFieldTypes();
+      fieldTypes.setMultiValued("sortedsetdv");
+      fieldTypes.setMultiValued("sortednumericdv");
+      fieldTypes.enableSorting("sorteddv");
+      fieldTypes.enableSorting("sortedsetdv");
+
+      Document2 doc = w.newDocument();
+      doc.addUniqueInt("id", 500);
+      doc.addStored("field", "some prepackaged text contents");
+      doc.addBinary("binarydv", new BytesRef("500"));
+      doc.addInt("numericdv", 500);
+      doc.addBinary("sorteddv", new BytesRef("500"));
+      doc.addBinary("sortedsetdv", new BytesRef("one"));
+      doc.addBinary("sortedsetdv", new BytesRef("two"));
+      doc.addInt("sortednumericdv", 4);
+      doc.addInt("sortednumericdv", 3);
       w.addDocument(doc);
-      doc = new Document();
-      doc.add(newStringField(random, "id", "501", Field.Store.NO));
-      doc.add(newField(random, "field", "some more contents", storedTextType));
-      doc.add(new BinaryDocValuesField("binarydv", new BytesRef("501")));
-      doc.add(new NumericDocValuesField("numericdv", 501));
-      doc.add(new SortedDocValuesField("sorteddv", new BytesRef("501")));
-      doc.add(new SortedSetDocValuesField("sortedsetdv", new BytesRef("two")));
-      doc.add(new SortedSetDocValuesField("sortedsetdv", new BytesRef("three")));
-      doc.add(new SortedNumericDocValuesField("sortednumericdv", 6));
-      doc.add(new SortedNumericDocValuesField("sortednumericdv", 1));
+
+      doc = w.newDocument();
+      doc.addUniqueInt("id", 501);
+      doc.addStored("field", "some more contents");
+      doc.addBinary("binarydv", new BytesRef("501"));
+      doc.addInt("numericdv", 501);
+      doc.addBinary("sorteddv", new BytesRef("501"));
+      doc.addBinary("sortedsetdv", new BytesRef("two"));
+      doc.addBinary("sortedsetdv", new BytesRef("three"));
+      doc.addInt("sortednumericdv", 6);
+      doc.addInt("sortednumericdv", 1);
       w.addDocument(doc);
-      w.deleteDocuments(new Term("id", "500"));
+      w.deleteDocuments(fieldTypes.newIntTerm("id", 500));
       w.close();
     }
 
@@ -966,33 +875,35 @@ public class TestIndexWriter extends Luc
                                                           new MockAnalyzer(random)).setMaxBufferedDocs(2);
             //conf.setInfoStream(log);
             w = new IndexWriter(dir, conf);
+            FieldTypes fieldTypes = w.getFieldTypes();
+            fieldTypes.enableSorting("sorteddv");
+            fieldTypes.enableSorting("sortedsetdv");
+            fieldTypes.setMultiValued("sortedsetdv");
 
-            Document doc = new Document();
-            Field idField = newStringField(random, "id", "", Field.Store.NO);
-            Field binaryDVField = new BinaryDocValuesField("binarydv", new BytesRef());
-            Field numericDVField = new NumericDocValuesField("numericdv", 0);
-            Field sortedDVField = new SortedDocValuesField("sorteddv", new BytesRef());
-            Field sortedSetDVField = new SortedSetDocValuesField("sortedsetdv", new BytesRef());
-            doc.add(idField);
-            doc.add(newField(random, "field", "some text contents", storedTextType));
-            doc.add(binaryDVField);
-            doc.add(numericDVField);
-            doc.add(sortedDVField);
-            doc.add(sortedSetDVField);
             for(int i=0;i<100;i++) {
               //log.println("\nTEST: i=" + i);
-              idField.setStringValue(Integer.toString(i));
-              binaryDVField.setBytesValue(new BytesRef(idField.stringValue()));
-              numericDVField.setLongValue(i);
-              sortedDVField.setBytesValue(new BytesRef(idField.stringValue()));
-              sortedSetDVField.setBytesValue(new BytesRef(idField.stringValue()));
+
+              BytesRef bytes = new BytesRef("" + i);
+
+              Document2 doc = w.newDocument();
+              doc.addLargeText("field", "some text contents");
+
+              doc.addInt("id", i);
+              doc.addBinary("binarydv", bytes);
+              doc.addInt("numericdv", i);
+              doc.addBinary("sorteddv", bytes);
+              doc.addBinary("sortedsetdv", bytes);
+
               int action = random.nextInt(100);
               if (action == 17) {
                 w.addIndexes(adder);
               } else if (action%30 == 0) {
                 w.deleteAll();
+                fieldTypes.enableSorting("sorteddv");
+                fieldTypes.enableSorting("sortedsetdv");
+                fieldTypes.setMultiValued("sortedsetdv");
               } else if (action%2 == 0) {
-                w.updateDocument(new Term("id", idField.stringValue()), doc);
+                w.updateDocument(fieldTypes.newIntTerm("id", i), doc);
               } else {
                 w.addDocument(doc);
               }
@@ -1277,13 +1188,11 @@ public class TestIndexWriter extends Luc
   public void testNoDocsIndex() throws Throwable {
     Directory dir = newDirectory();
     IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
-    writer.addDocument(new Document());
+    writer.addDocument(writer.newDocument());
     writer.close();
-
     dir.close();
   }
 
-
   public void testDeleteUnusedFiles() throws Exception {
     assumeFalse("test relies on exact filenames", Codec.getDefault() instanceof SimpleTextCodec);
     for(int iter=0;iter<2;iter++) {
@@ -1302,8 +1211,8 @@ public class TestIndexWriter extends Luc
             .setMergePolicy(mergePolicy)
             .setUseCompoundFile(true)
       );
-      Document doc = new Document();
-      doc.add(newTextField("field", "go", Field.Store.NO));
+      Document2 doc = w.newDocument();
+      doc.addLargeText("field", "go");
       w.addDocument(doc);
       DirectoryReader r;
       if (iter == 0) {
@@ -1386,17 +1295,17 @@ public class TestIndexWriter extends Luc
     }
     IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
                                                 .setIndexDeletionPolicy(new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy())));
+    FieldTypes fieldTypes = writer.getFieldTypes();
+    
     SnapshotDeletionPolicy sdp = (SnapshotDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
 
     // First commit
-    Document doc = new Document();
-
-    FieldType customType = new FieldType(TextField.TYPE_STORED);
-    customType.setStoreTermVectors(true);
-    customType.setStoreTermVectorPositions(true);
-    customType.setStoreTermVectorOffsets(true);
+    fieldTypes.enableTermVectors("c");
+    fieldTypes.enableTermVectorPositions("c");
+    fieldTypes.enableTermVectorOffsets("c");
     
-    doc.add(newField("c", "val", customType));
+    Document2 doc = writer.newDocument();
+    doc.addLargeText("c", "val");
     writer.addDocument(doc);
     writer.commit();
     assertEquals(1, DirectoryReader.listCommits(dir).size());
@@ -1405,8 +1314,8 @@ public class TestIndexWriter extends Luc
     IndexCommit id = sdp.snapshot();
 
     // Second commit - now KeepOnlyLastCommit cannot delete the prev commit.
-    doc = new Document();
-    doc.add(newField("c", "val", customType));
+    doc = writer.newDocument();
+    doc.addLargeText("c", "val");
     writer.addDocument(doc);
     writer.commit();
     assertEquals(2, DirectoryReader.listCommits(dir).size());
@@ -1443,6 +1352,7 @@ public class TestIndexWriter extends Luc
                                                 .setMaxBufferedDocs(2)
                                                 .setMergePolicy(newLogMergePolicy())
                                                 .setUseCompoundFile(false));
+    FieldTypes fieldTypes = writer.getFieldTypes();
     String[] files = dir.listAll();
 
     // Creating over empty dir should not create any files,
@@ -1456,13 +1366,13 @@ public class TestIndexWriter extends Luc
       extraFileCount = 0;
     }
 
-    Document doc = new Document();
-    FieldType customType = new FieldType(TextField.TYPE_STORED);
-    customType.setStoreTermVectors(true);
-    customType.setStoreTermVectorPositions(true);
-    customType.setStoreTermVectorOffsets(true);
+    fieldTypes.enableTermVectors("c");
+    fieldTypes.enableTermVectorPositions("c");
+    fieldTypes.enableTermVectorOffsets("c");
+
+    Document2 doc = writer.newDocument();
     // create as many files as possible
-    doc.add(newField("c", "val", customType));
+    doc.addLargeText("c", "val");
     writer.addDocument(doc);
     // Adding just one document does not call flush yet.
     int computedExtraFileCount = 0;
@@ -1475,8 +1385,8 @@ public class TestIndexWriter extends Luc
     }
     assertEquals("only the stored and term vector files should exist in the directory", extraFileCount, computedExtraFileCount);
 
-    doc = new Document();
-    doc.add(newField("c", "val", customType));
+    doc = writer.newDocument();
+    doc.addLargeText("c", "val");
     writer.addDocument(doc);
 
     // The second document should cause a flush.
@@ -1499,17 +1409,18 @@ public class TestIndexWriter extends Luc
     IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
                                            .setMaxBufferedDocs(2));
 
-    Document doc = new Document();
-    FieldType customType = new FieldType(TextField.TYPE_STORED);
-    customType.setStoreTermVectors(true);
-    customType.setStoreTermVectorPositions(true);
-    customType.setStoreTermVectorOffsets(true);
-    doc.add(newField("c", "val", customType));
+    FieldTypes fieldTypes = w.getFieldTypes();
+    fieldTypes.enableTermVectors("c");
+    fieldTypes.enableTermVectorPositions("c");
+    fieldTypes.enableTermVectorOffsets("c");
+
+    Document2 doc = w.newDocument();
+    doc.addLargeText("c", "val");
     w.addDocument(doc);
     w.addDocument(doc);
     IndexWriter w2 = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
-                                            .setMaxBufferedDocs(2)
-                                            .setOpenMode(OpenMode.CREATE));
+                                     .setMaxBufferedDocs(2)
+                                     .setOpenMode(OpenMode.CREATE));
 
     w2.close();
     // If we don't do that, the test fails on Windows
@@ -1531,25 +1442,21 @@ public class TestIndexWriter extends Luc
     IndexWriter indexWriter = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
                                                      .setRAMBufferSizeMB(0.01)
                                                      .setMergePolicy(newLogMergePolicy()));
+    FieldTypes fieldTypes = indexWriter.getFieldTypes();
     indexWriter.getConfig().getMergePolicy().setNoCFSRatio(0.0);
 
     String BIG="alskjhlaksjghlaksjfhalksvjepgjioefgjnsdfjgefgjhelkgjhqewlrkhgwlekgrhwelkgjhwelkgrhwlkejg";
     BIG=BIG+BIG+BIG+BIG;
 
-    FieldType customType = new FieldType(TextField.TYPE_STORED);
-    customType.setOmitNorms(true);
-    FieldType customType2 = new FieldType(TextField.TYPE_STORED);
-    customType2.setTokenized(false);
-    FieldType customType3 = new FieldType(TextField.TYPE_STORED);
-    customType3.setTokenized(false);
-    customType3.setOmitNorms(true);
-    
+    fieldTypes.disableNorms("id");
+    fieldTypes.disableNorms("str3");
+
     for (int i=0; i<2; i++) {
-      Document doc = new Document();
-      doc.add(new Field("id", Integer.toString(i)+BIG, customType3));
-      doc.add(new Field("str", Integer.toString(i)+BIG, customType2));
-      doc.add(new Field("str2", Integer.toString(i)+BIG, storedTextType));
-      doc.add(new Field("str3", Integer.toString(i)+BIG, customType));
+      Document2 doc = indexWriter.newDocument();
+      doc.addAtom("id", Integer.toString(i)+BIG);
+      doc.addAtom("str", Integer.toString(i)+BIG);
+      doc.addLargeText("str2", Integer.toString(i)+BIG);
+      doc.addLargeText("str3", Integer.toString(i)+BIG);
       indexWriter.addDocument(doc);
     }
 
@@ -1620,12 +1527,12 @@ public class TestIndexWriter extends Luc
 
     char[] chars = new char[DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8];
     Arrays.fill(chars, 'x');
-    Document doc = new Document();
+    Document2 doc = w.newDocument();
     final String bigTerm = new String(chars);
 
     // This contents produces a too-long term:
     String contents = "abc xyz x" + bigTerm + " another term";
-    doc.add(new TextField("content", contents, Field.Store.NO));
+    doc.addLargeText("content", contents);
     try {
       w.addDocument(doc);
       fail("should have hit exception");
@@ -1634,8 +1541,8 @@ public class TestIndexWriter extends Luc
     }
 
     // Make sure we can add another normal document
-    doc = new Document();
-    doc.add(new TextField("content", "abc bbb ccc", Field.Store.NO));
+    doc = w.newDocument();
+    doc.addLargeText("content", "abc bbb ccc");
     w.addDocument(doc);
 
     // So we remove the deleted doc:
@@ -1659,24 +1566,23 @@ public class TestIndexWriter extends Luc
 
     // Make sure we can add a document with exactly the
     // maximum length term, and search on that term:
-    doc = new Document();
-    FieldType customType = new FieldType(TextField.TYPE_NOT_STORED);
-    customType.setTokenized(false);
-    Field contentField = new Field("content", "", customType);
-    doc.add(contentField);
 
     w = new RandomIndexWriter(random(), dir);
 
-    contentField.setStringValue("other");
+    doc = w.newDocument();
+    doc.addAtom("content", "other");
     w.addDocument(doc);
 
-    contentField.setStringValue("term");
+    doc = w.newDocument();
+    doc.addAtom("content", "term");
     w.addDocument(doc);
 
-    contentField.setStringValue(bigTerm);
+    doc = w.newDocument();
+    doc.addAtom("content", bigTerm);
     w.addDocument(doc);
 
-    contentField.setStringValue("zzz");
+    doc = w.newDocument();
+    doc.addAtom("content", "zzz");
     w.addDocument(doc);
 
     reader = w.getReader();
@@ -1692,8 +1598,8 @@ public class TestIndexWriter extends Luc
     MockDirectoryWrapper d = new MockDirectoryWrapper(random(), new RAMDirectory());
     d.setEnableVirusScanner(false); // needs for files to actually be deleted
     IndexWriter w = new IndexWriter(d, new IndexWriterConfig(new MockAnalyzer(random())));
-    Document doc = new Document();
     for(int i = 0; i < 20; i++) {
+      Document2 doc = w.newDocument();
       for(int j = 0; j < 100; ++j) {
         w.addDocument(doc);
       }
@@ -1714,8 +1620,8 @@ public class TestIndexWriter extends Luc
   public void testNRTReaderVersion() throws Exception {
     Directory d = new MockDirectoryWrapper(random(), new RAMDirectory());
     IndexWriter w = new IndexWriter(d, new IndexWriterConfig(new MockAnalyzer(random())));
-    Document doc = new Document();
-    doc.add(newStringField("id", "0", Field.Store.YES));
+    Document2 doc = w.newDocument();
+    doc.addAtom("id", "0");
     w.addDocument(doc);
     DirectoryReader r = w.getReader();
     long version = r.getVersion();
@@ -1759,19 +1665,18 @@ public class TestIndexWriter extends Luc
     IndexWriter w = new IndexWriter(dir,
                                     new IndexWriterConfig(new MockAnalyzer(random())));
 
-    FieldType docsAndFreqs = new FieldType(TextField.TYPE_NOT_STORED);
-    docsAndFreqs.setIndexOptions(IndexOptions.DOCS_AND_FREQS);
+    FieldTypes fieldTypes = w.getFieldTypes();
+    fieldTypes.disableHighlighting("field");
+    fieldTypes.setIndexOptions("field", IndexOptions.DOCS_AND_FREQS);
 
-    FieldType docsOnly = new FieldType(TextField.TYPE_NOT_STORED);
-    docsOnly.setIndexOptions(IndexOptions.DOCS);
-
-    Document doc = new Document();
-    doc.add(new Field("field", "a b c", docsAndFreqs));
+    Document2 doc = w.newDocument();
+    doc.addLargeText("field", "a b c");
     w.addDocument(doc);
     w.addDocument(doc);
 
-    doc = new Document();
-    doc.add(new Field("field", "a b c", docsOnly));
+    doc = w.newDocument();
+    fieldTypes.setIndexOptions("field", IndexOptions.DOCS);
+    doc.addLargeText("field", "a b c");
     w.addDocument(doc);
     w.close();
     dir.close();
@@ -1782,10 +1687,9 @@ public class TestIndexWriter extends Luc
     IndexWriter w = new IndexWriter(dir,
                                     new IndexWriterConfig(new MockAnalyzer(random())));
 
-    final List<Document> docs = new ArrayList<>();
-    docs.add(new Document());
-    w.updateDocuments(new Term("foo", "bar"),
-                      docs);
+    final List<Document2> docs = new ArrayList<>();
+    docs.add(w.newDocument());
+    w.updateDocuments(new Term("foo", "bar"), docs);
     w.close();
     dir.close();
   }
@@ -1834,7 +1738,7 @@ public class TestIndexWriter extends Luc
                                     new IndexWriterConfig(new MockAnalyzer(random())));
 
     w.commit();
-    w.addDocument(new Document());
+    w.addDocument(w.newDocument());
     w.prepareCommit();
     w.rollback();
     assertTrue(DirectoryReader.indexExists(dir));
@@ -1851,7 +1755,7 @@ public class TestIndexWriter extends Luc
     Directory dir = newDirectory();
     IndexWriter iw = new IndexWriter(dir, 
         newIndexWriterConfig(new MockAnalyzer(random())));
-    iw.addDocument(new Document());
+    iw.addDocument(iw.newDocument());
     iw.close();
     try {
       // Create my own random file:
@@ -1879,9 +1783,12 @@ public class TestIndexWriter extends Luc
       }
     };
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir, a);
-    Document doc = new Document();
-    doc.add(new TextField("body", "just a", Field.Store.NO));
-    doc.add(new TextField("body", "test of gaps", Field.Store.NO));
+    FieldTypes fieldTypes = iw.getFieldTypes();
+    fieldTypes.setMultiValued("body");
+
+    Document2 doc = iw.newDocument();
+    doc.addLargeText("body", "just a");
+    doc.addLargeText("body", "test of gaps");
     iw.addDocument(doc);
     IndexReader ir = iw.getReader();
     iw.close();
@@ -1910,9 +1817,12 @@ public class TestIndexWriter extends Luc
       }
     };
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir, a);
-    Document doc = new Document();
-    doc.add(new TextField("body", "just a foobar", Field.Store.NO));
-    doc.add(new TextField("body", "test of gaps", Field.Store.NO));
+    FieldTypes fieldTypes = iw.getFieldTypes();
+    fieldTypes.setMultiValued("body");
+
+    Document2 doc = iw.newDocument();
+    doc.addLargeText("body", "just a foobar");
+    doc.addLargeText("body", "test of gaps");
     iw.addDocument(doc);
     IndexReader ir = iw.getReader();
     iw.close();
@@ -1993,14 +1903,14 @@ public class TestIndexWriter extends Luc
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwConf);
     // add 3 good docs
     for (int i = 0; i < 3; i++) {
-      Document doc = new Document();
-      doc.add(new StringField("id", Integer.toString(i), Field.Store.NO));
+      Document2 doc = iw.newDocument();
+      doc.addAtom("id", Integer.toString(i));
       iw.addDocument(doc);
     }
     // add broken doc
     try {
-      Document broke = new Document();
-      broke.add(newTextField("test", "broken", Field.Store.NO));
+      Document2 broke = iw.newDocument();
+      broke.addLargeText("test", "broken");
       iw.addDocument(broke);
       fail();
     } catch (NullPointerException expected) {}
@@ -2017,15 +1927,16 @@ public class TestIndexWriter extends Luc
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
     // add 3 good docs
     for (int i = 0; i < 3; i++) {
-      Document doc = new Document();
-      doc.add(new StringField("id", Integer.toString(i), Field.Store.NO));
+      Document2 doc = iw.newDocument();
+      doc.addAtom("id", Integer.toString(i));
       iw.addDocument(doc);
     }
     // add broken doc
     try {
       iw.addDocument(null);
       fail();
-    } catch (NullPointerException expected) {}
+    } catch (NullPointerException expected) {
+    }
     // ensure good docs are still ok
     IndexReader ir = iw.getReader();
     assertEquals(3, ir.numDocs());
@@ -2039,8 +1950,8 @@ public class TestIndexWriter extends Luc
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
     // add 3 good docs
     for (int i = 0; i < 3; i++) {
-      Document doc = new Document();
-      doc.add(new StringField("id", Integer.toString(i), Field.Store.NO));
+      Document2 doc = iw.newDocument();
+      doc.addAtom("id", Integer.toString(i));
       iw.addDocument(doc);
     }
     // add broken doc block
@@ -2061,8 +1972,8 @@ public class TestIndexWriter extends Luc
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
     // add 3 good docs
     for (int i = 0; i < 3; i++) {
-      Document doc = new Document();
-      doc.add(new StringField("id", Integer.toString(i), Field.Store.NO));
+      Document2 doc = iw.newDocument();
+      doc.addAtom("id", Integer.toString(i));
       iw.addDocument(doc);
     }
     // add broken doc
@@ -2134,17 +2045,15 @@ public class TestIndexWriter extends Luc
     int iters = atLeast(100);
     int docCount = 0;
     int docId = 0;
-    Set<String> liveIds = new HashSet<>();
+    Set<Integer> liveIds = new HashSet<>();
     for (int i = 0; i < iters; i++) {
       List<Iterable<IndexableField>> docs = new ArrayList<>();
-      FieldType ft = new FieldType(TextField.TYPE_NOT_STORED);
-      FieldType idFt = new FieldType(TextField.TYPE_STORED);
       
       int numDocs = atLeast(4);
       for (int j = 0; j < numDocs; j++) {
-        Document doc = new Document();
-        doc.add(newField("id", ""+ (docId++), idFt));
-        doc.add(newField("foo", TestUtil.randomSimpleString(random()), ft));
+        Document2 doc = w.newDocument();
+        doc.addUniqueInt("id", docId++);
+        doc.addLargeText("foo", TestUtil.randomSimpleString(random()));
         docs.add(doc);
       }
       boolean success = false;
@@ -2157,7 +2066,7 @@ public class TestIndexWriter extends Luc
         if (success) {
           docCount += docs.size();
           for (Iterable<IndexableField> indexDocument : docs) {
-            liveIds.add(((Document) indexDocument).get("id"));  
+            liveIds.add(((Document2) indexDocument).getInt("id"));  
           }
         }
       }
@@ -2171,7 +2080,7 @@ public class TestIndexWriter extends Luc
       int maxDoc = ar.maxDoc();
       for (int i = 0; i < maxDoc; i++) {
         if (liveDocs == null || liveDocs.get(i)) {
-          assertTrue(liveIds.remove(ar.document(i).get("id")));
+          assertTrue(liveIds.remove(ar.document(i).getInt("id")));
         }
       }
     }
@@ -2315,8 +2224,8 @@ public class TestIndexWriter extends Luc
     Directory dir = newDirectory();
     IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
     assertTrue(writer.hasUncommittedChanges());  // this will be true because a commit will create an empty index
-    Document doc = new Document();
-    doc.add(newTextField("myfield", "a b c", Field.Store.NO));
+    Document2 doc = writer.newDocument();
+    doc.addLargeText("myfield", "a b c");
     writer.addDocument(doc);
     assertTrue(writer.hasUncommittedChanges());
 
@@ -2329,8 +2238,8 @@ public class TestIndexWriter extends Luc
     writer.addDocument(doc);
     assertTrue(writer.hasUncommittedChanges());
     writer.commit();
-    doc = new Document();
-    doc.add(newStringField("id", "xyz", Field.Store.YES));
+    doc = writer.newDocument();
+    doc.addAtom("id", "xyz");
     writer.addDocument(doc);
     assertTrue(writer.hasUncommittedChanges());
 
@@ -2353,6 +2262,8 @@ public class TestIndexWriter extends Luc
 
     writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
     assertFalse(writer.hasUncommittedChanges());
+    doc = writer.newDocument();
+    doc.addAtom("id", "xyz");
     writer.addDocument(doc);
     assertTrue(writer.hasUncommittedChanges());
 
@@ -2393,8 +2304,8 @@ public class TestIndexWriter extends Luc
     Directory dir = newDirectory();
     IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
     IndexWriter w = new IndexWriter(dir, iwc);
-    Document doc = new Document();
-    doc.add(new TextField("a", "foo", Field.Store.NO));
+    Document2 doc = w.newDocument();
+    doc.addLargeText("a", "foo");
     w.addDocument(doc);
 
     // Should not delete the document; with LUCENE-5239 the
@@ -2419,9 +2330,11 @@ public class TestIndexWriter extends Luc
     IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
     iwc.setMergePolicy(newLogMergePolicy());
     IndexWriter iwriter = new IndexWriter(directory, iwc);
-    Document doc = new Document();
-    doc.add(new SortedDocValuesField("dv", new BytesRef("foo!")));
-    doc.add(new SortedDocValuesField("dv", new BytesRef("bar!")));
+    FieldTypes fieldTypes = iwriter.getFieldTypes();
+    fieldTypes.enableSorting("dv");
+    Document2 doc = iwriter.newDocument();
+    doc.addBinary("dv", new BytesRef("foo!"));
+    doc.addBinary("dv", new BytesRef("bar!"));
     try {
       iwriter.addDocument(doc);
       fail("didn't hit expected exception");
@@ -2437,8 +2350,8 @@ public class TestIndexWriter extends Luc
   public void testDoubleClose() throws IOException {
     Directory dir = newDirectory();
     IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
-    Document doc = new Document();
-    doc.add(new SortedDocValuesField("dv", new BytesRef("foo!")));
+    Document2 doc = w.newDocument();
+    doc.addBinary("dv", new BytesRef("foo!"));
     w.addDocument(doc);
     w.close();
     // Close again should have no effect
@@ -2449,8 +2362,8 @@ public class TestIndexWriter extends Luc
   public void testRollbackThenClose() throws IOException {
     Directory dir = newDirectory();
     IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
-    Document doc = new Document();
-    doc.add(new SortedDocValuesField("dv", new BytesRef("foo!")));
+    Document2 doc = w.newDocument();
+    doc.addBinary("dv", new BytesRef("foo!"));
     w.addDocument(doc);
     w.rollback();
     // Close after rollback should have no effect
@@ -2461,8 +2374,8 @@ public class TestIndexWriter extends Luc
   public void testCloseThenRollback() throws IOException {
     Directory dir = newDirectory();
     IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
-    Document doc = new Document();
-    doc.add(new SortedDocValuesField("dv", new BytesRef("foo!")));
+    Document2 doc = w.newDocument();
+    doc.addBinary("dv", new BytesRef("foo!"));
     w.addDocument(doc);
     w.close();
     // Rollback after close should have no effect
@@ -2516,8 +2429,8 @@ public class TestIndexWriter extends Luc
         }
       });
     IndexWriter w = new IndexWriter(dir, iwc);
-    Document doc = new Document();
-    doc.add(new SortedDocValuesField("dv", new BytesRef("foo!")));
+    Document2 doc = w.newDocument();
+    doc.addShortText("dv", "foo!");
     w.addDocument(doc);
     w.commit();
     w.addDocument(doc);
@@ -2550,8 +2463,8 @@ public class TestIndexWriter extends Luc
     iwc.setMergePolicy(lmp);
 
     RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
-    Document doc = new Document();
-    doc.add(new TextField("a", "foo", Field.Store.NO));
+    Document2 doc = w.newDocument();
+    doc.addLargeText("a", "foo");
     w.addDocument(doc);
     w.commit();
     w.addDocument(doc);
@@ -2566,6 +2479,8 @@ public class TestIndexWriter extends Luc
     }
 
     w = new RandomIndexWriter(random(), dir);
+    doc = w.newDocument();
+    doc.addLargeText("a", "foo");
     w.addDocument(doc);
     w.close();
     r.close();
@@ -2633,7 +2548,7 @@ public class TestIndexWriter extends Luc
   public void testIds() throws Exception {
     Directory d = newDirectory();
     IndexWriter w = new IndexWriter(d, newIndexWriterConfig(new MockAnalyzer(random())));
-    w.addDocument(new Document());
+    w.addDocument(w.newDocument());
     w.close();
     
     SegmentInfos sis = SegmentInfos.readLatestCommit(d);
@@ -2673,8 +2588,8 @@ public class TestIndexWriter extends Luc
   public void testEmptyNorm() throws Exception {
     Directory d = newDirectory();
     IndexWriter w = new IndexWriter(d, newIndexWriterConfig(new MockAnalyzer(random())));
-    Document doc = new Document();
-    doc.add(new TextField("foo", new CannedTokenStream()));
+    Document2 doc = w.newDocument();
+    doc.addLargeText("foo", new CannedTokenStream());
     w.addDocument(doc);
     w.commit();
     w.close();

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java?rev=1640053&r1=1640052&r2=1640053&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java Mon Nov 17 00:43:44 2014
@@ -24,6 +24,7 @@ import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.lucene.analysis.*;
+import org.apache.lucene.document.Document2;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
@@ -241,7 +242,7 @@ public class TestIndexWriterCommit exten
 
     );
     for(int j=0;j<1470;j++) {
-      TestIndexWriter.addDocWithIndex(writer, j);
+      TestIndexWriter.addDocWithIndex(writer, 30+j);
     }
     long midDiskUsage = dir.getMaxUsedSizeInBytes();
     dir.resetMaxUsedSizeInBytes();
@@ -359,16 +360,14 @@ public class TestIndexWriterCommit exten
           @Override
           public void run() {
             try {
-              final Document doc = new Document();
               DirectoryReader r = DirectoryReader.open(dir);
-              Field f = newStringField("f", "", Field.Store.NO);
-              doc.add(f);
               int count = 0;
               do {
                 if (failed.get()) break;
                 for(int j=0;j<10;j++) {
                   final String s = finalI + "_" + String.valueOf(count++);
-                  f.setStringValue(s);
+                  Document2 doc = w.newDocument();
+                  doc.addAtom("f", s);
                   w.addDocument(doc);
                   w.commit();
                   DirectoryReader r2 = DirectoryReader.openIfChanged(r);
@@ -441,8 +440,7 @@ public class TestIndexWriterCommit exten
 
     IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
                                            .setIndexDeletionPolicy(NoDeletionPolicy.INSTANCE));
-    Document doc = new Document();
-    w.addDocument(doc);
+    w.addDocument(w.newDocument());
 
     // commit to "first"
     Map<String,String> commitData = new HashMap<>();
@@ -451,7 +449,7 @@ public class TestIndexWriterCommit exten
     w.commit();
 
     // commit to "second"
-    w.addDocument(doc);
+    w.addDocument(w.newDocument());
     commitData.put("tag", "second");
     w.setCommitData(commitData);
     w.close();
@@ -474,7 +472,7 @@ public class TestIndexWriterCommit exten
     assertEquals(1, w.numDocs());
 
     // commit IndexWriter to "third"
-    w.addDocument(doc);
+    w.addDocument(w.newDocument());
     commitData.put("tag", "third");
     w.setCommitData(commitData);
     w.close();
@@ -692,7 +690,7 @@ public class TestIndexWriterCommit exten
   public void testPrepareCommitThenClose() throws Exception {
     Directory dir = newDirectory();
     IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
-    w.addDocument(new Document());
+    w.addDocument(w.newDocument());
     w.prepareCommit();
     try {
       w.close();

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java?rev=1640053&r1=1640052&r2=1640053&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java Mon Nov 17 00:43:44 2014
@@ -25,6 +25,7 @@ import java.util.Set;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.document.Document2;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain;
@@ -327,13 +328,14 @@ public class TestIndexWriterConfig exten
     // Change to true:
     w.getConfig().setUseCompoundFile(true);
 
-    Document doc = new Document();
-    doc.add(newStringField("field", "foo", Store.NO));
+    Document2 doc = w.newDocument();
+    doc.addAtom("field", "foo");
     w.addDocument(doc);
     w.commit();
     assertTrue("Expected CFS after commit", w.newestSegment().info.getUseCompoundFile());
     
-    doc.add(newStringField("field", "foo", Store.NO));
+    doc = w.newDocument();
+    doc.addAtom("field", "foo");
     w.addDocument(doc);
     w.commit();
     w.forceMerge(1);