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/30 12:07:19 UTC

svn commit: r1642535 [19/19] - in /lucene/dev/branches/lucene6005/lucene: analysis/common/src/java/org/apache/lucene/collation/ analysis/common/src/test/org/apache/lucene/analysis/core/ analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/...

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseFieldInfoFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseFieldInfoFormatTestCase.java?rev=1642535&r1=1642534&r2=1642535&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseFieldInfoFormatTestCase.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseFieldInfoFormatTestCase.java Sun Nov 30 11:07:09 2014
@@ -23,11 +23,7 @@ import java.util.Random;
 import java.util.Set;
 
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.document.Document2;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.FieldType;
-import org.apache.lucene.document.StoredField;
-import org.apache.lucene.document.TextField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.StringHelper;
@@ -51,7 +47,7 @@ public abstract class BaseFieldInfoForma
     SegmentInfo segmentInfo = newSegmentInfo(dir, "_123");
     FieldInfos.Builder builder = new FieldInfos.Builder();
     FieldInfo fi = builder.getOrAdd("field");
-    fi.setIndexOptions(TextField.TYPE_STORED.indexOptions());
+    fi.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
     addAttributes(fi);
     FieldInfos infos = builder.finish();
     codec.fieldInfosFormat().write(dir, segmentInfo, "", infos, IOContext.DEFAULT);
@@ -82,17 +78,33 @@ public abstract class BaseFieldInfoForma
     }
     FieldInfos.Builder builder = new FieldInfos.Builder();
     for (String field : fieldNames) {
-      IndexableFieldType fieldType = randomFieldType(random());
       FieldInfo fi = builder.getOrAdd(field);
-      IndexOptions indexOptions = fieldType.indexOptions();
+      IndexOptions indexOptions;
+      boolean omitNorms = false;
+      if (random().nextBoolean()) {
+        IndexOptions values[] = IndexOptions.values();
+        indexOptions = values[random().nextInt(values.length)];
+        omitNorms = random().nextBoolean();
+      } else {
+        indexOptions = IndexOptions.NONE;
+      }
+
+      DocValuesType docValuesType;
+      if (random().nextBoolean()) {
+        DocValuesType values[] = getDocValuesTypes();
+        docValuesType = values[random().nextInt(values.length)];
+      } else {
+        docValuesType = DocValuesType.NONE;
+      }
+
       if (indexOptions != IndexOptions.NONE) {
         fi.setIndexOptions(indexOptions);
-        if (fieldType.omitNorms()) {      
+        if (omitNorms) {
           fi.setOmitsNorms();
         }
       }
-      fi.setDocValuesType(fieldType.docValuesType());
-      if (fieldType.indexOptions() != IndexOptions.NONE && fieldType.indexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0) {
+      fi.setDocValuesType(docValuesType);
+      if (indexOptions != IndexOptions.NONE && indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0) {
         if (random().nextBoolean()) {
           fi.setStorePayloads();
         }
@@ -106,34 +118,6 @@ public abstract class BaseFieldInfoForma
     dir.close();
   }
   
-  private final IndexableFieldType randomFieldType(Random r) {
-    FieldType type = new FieldType();
-    
-    if (r.nextBoolean()) {
-      IndexOptions values[] = IndexOptions.values();
-      type.setIndexOptions(values[r.nextInt(values.length)]);
-      type.setOmitNorms(r.nextBoolean());
-      
-      if (r.nextBoolean()) {
-        type.setStoreTermVectors(true);
-        if (type.indexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0) {
-          type.setStoreTermVectorPositions(r.nextBoolean());
-          type.setStoreTermVectorOffsets(r.nextBoolean());
-          if (type.storeTermVectorPositions()) {
-            type.setStoreTermVectorPayloads(r.nextBoolean());
-          }
-        }
-      }
-    }
-    
-    if (r.nextBoolean()) {
-      DocValuesType values[] = getDocValuesTypes();
-      type.setDocValuesType(values[r.nextInt(values.length)]);
-    }
-        
-    return type;
-  }
-  
   /** 
    * Hook to add any codec attributes to fieldinfo
    * instances added in this test.
@@ -180,7 +164,7 @@ public abstract class BaseFieldInfoForma
   }
   
   @Override
-  protected void addRandomFields(Document2 doc) {
+  protected void addRandomFields(Document doc) {
     doc.addStored("foobar", TestUtil.randomSimpleString(random()));
   }
 

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java?rev=1642535&r1=1642534&r2=1642535&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java Sun Nov 30 11:07:09 2014
@@ -32,7 +32,6 @@ import java.util.Set;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
-import org.apache.lucene.document.Document2;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.FieldTypes;
 import org.apache.lucene.store.Directory;
@@ -138,7 +137,7 @@ abstract class BaseIndexFileFormatTestCa
   }
 
   /** Add random fields to the provided document. */
-  protected abstract void addRandomFields(Document2 doc);
+  protected abstract void addRandomFields(Document doc);
 
   private Map<String, Long> bytesUsedByExtension(Directory d) throws IOException {
     Map<String, Long> bytesUsedByExtension = new HashMap<>();
@@ -185,7 +184,7 @@ abstract class BaseIndexFileFormatTestCa
 
     final int numDocs = atLeast(500);
     for (int i = 0; i < numDocs; ++i) {
-      Document2 d = w.newDocument();
+      Document d = w.newDocument();
       addRandomFields(d);
       w.addDocument(d);
     }
@@ -234,7 +233,7 @@ abstract class BaseIndexFileFormatTestCa
     final int numDocs = atLeast(10000);
     LeafReader reader1 = null;
     for (int i = 0; i < numDocs; ++i) {
-      Document2 d = w.newDocument();
+      Document d = w.newDocument();
       addRandomFields(d);
       w.addDocument(d);
       if (i == 100) {

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseMergePolicyTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseMergePolicyTestCase.java?rev=1642535&r1=1642534&r2=1642535&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseMergePolicyTestCase.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseMergePolicyTestCase.java Sun Nov 30 11:07:09 2014
@@ -18,7 +18,6 @@ package org.apache.lucene.index;
  */
 
 import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java?rev=1642535&r1=1642534&r2=1642535&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java Sun Nov 30 11:07:09 2014
@@ -25,11 +25,7 @@ import java.util.Random;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenizer;
-import org.apache.lucene.document.Document2;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.document.TextField;
 import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.TermStatistics;
 import org.apache.lucene.search.similarities.Similarity;
@@ -249,7 +245,7 @@ public abstract class BaseNormsFormatTes
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
     
     for (int i = 0; i < numDocs; i++) {
-      Document2 doc = writer.newDocument();
+      Document doc = writer.newDocument();
       doc.addAtom("id", Integer.toString(i));
       long value = norms[i];
       doc.addLargeText("stored", Long.toString(value));
@@ -328,7 +324,7 @@ public abstract class BaseNormsFormatTes
   }
 
   @Override
-  protected void addRandomFields(Document2 doc) {
+  protected void addRandomFields(Document doc) {
     // TODO: improve
     doc.addLargeText("foobar", TestUtil.randomSimpleString(random()));
   }
@@ -372,7 +368,7 @@ public abstract class BaseNormsFormatTes
     int numDocs = atLeast(1000);
     List<Integer> toDelete = new ArrayList<>();
     for(int i=0;i<numDocs;i++) {
-      Document2 doc = w.newDocument();
+      Document doc = w.newDocument();
       if (random().nextInt(5) == 1) {
         toDelete.add(i);
         doc.addAtom("id", ""+i);

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java?rev=1642535&r1=1642534&r2=1642535&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java Sun Nov 30 11:07:09 2014
@@ -42,10 +42,7 @@ import org.apache.lucene.codecs.FieldsPr
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.asserting.AssertingCodec;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
-import org.apache.lucene.document.Document2;
 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.index.TermsEnum.SeekStatus;
 import org.apache.lucene.store.Directory;
@@ -1415,7 +1412,7 @@ public abstract class BasePostingsFormat
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
     FieldTypes fieldTypes = iw.getFieldTypes();
     fieldTypes.disableExistsFilters();
-    Document2 doc = iw.newDocument();
+    Document doc = iw.newDocument();
     doc.addAtom("", "something");
     iw.addDocument(doc);
     DirectoryReader ir = iw.getReader();
@@ -1442,7 +1439,7 @@ public abstract class BasePostingsFormat
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
     FieldTypes fieldTypes = iw.getFieldTypes();
     fieldTypes.disableExistsFilters();
-    Document2 doc = iw.newDocument();
+    Document doc = iw.newDocument();
     doc.addAtom("", "");
     iw.addDocument(doc);
     DirectoryReader ir = iw.getReader();
@@ -1472,7 +1469,7 @@ public abstract class BasePostingsFormat
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
     FieldTypes fieldTypes = iw.getFieldTypes();
     fieldTypes.disableExistsFilters();
-    Document2 doc = iw.newDocument();
+    Document doc = iw.newDocument();
     iw.addDocument(doc);
     doc.addAtom("ghostField", "something");
     iw.addDocument(doc);
@@ -1701,7 +1698,7 @@ public abstract class BasePostingsFormat
     int bytesToIndex = atLeast(100) * 1024;
     int bytesIndexed = 0;
     while (bytesIndexed < bytesToIndex) {
-      Document2 doc = docs.nextDoc();
+      Document doc = docs.nextDoc();
       w.addDocument(doc);
       bytesIndexed += RamUsageTester.sizeOf(doc);
     }
@@ -1741,7 +1738,7 @@ public abstract class BasePostingsFormat
   }
 
   @Override
-  protected void addRandomFields(Document2 doc) {
+  protected void addRandomFields(Document doc) {
     FieldTypes fieldTypes = doc.getFieldTypes();
     for (IndexOptions opts : IndexOptions.values()) {
       if (opts == IndexOptions.NONE) {

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseSegmentInfoFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseSegmentInfoFormatTestCase.java?rev=1642535&r1=1642534&r2=1642535&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseSegmentInfoFormatTestCase.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseSegmentInfoFormatTestCase.java Sun Nov 30 11:07:09 2014
@@ -25,9 +25,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.document.Document2;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.StoredField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.StringHelper;
@@ -190,7 +188,7 @@ public abstract class BaseSegmentInfoFor
   }
   
   @Override
-  protected void addRandomFields(Document2 doc) {
+  protected void addRandomFields(Document doc) {
     doc.addStored("foobar", TestUtil.randomSimpleString(random()));
   }
 

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java?rev=1642535&r1=1642534&r2=1642535&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java Sun Nov 30 11:07:09 2014
@@ -33,17 +33,8 @@ import org.apache.lucene.analysis.MockAn
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
-import org.apache.lucene.document.Document2;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field.Store;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.FieldType.NumericType;
-import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.FieldTypes;
-import org.apache.lucene.document.NumericDocValuesField;
-import org.apache.lucene.document.StoredField;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.document.TextField;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.NumericRangeQuery;
 import org.apache.lucene.search.Query;
@@ -56,7 +47,6 @@ import org.apache.lucene.store.MockDirec
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.TestUtil;
 import com.carrotsearch.randomizedtesting.generators.RandomInts;
-import com.carrotsearch.randomizedtesting.generators.RandomPicks;
 
 /**
  * Base class aiming at testing {@link StoredFieldsFormat stored fields formats}.
@@ -67,7 +57,7 @@ import com.carrotsearch.randomizedtestin
 public abstract class BaseStoredFieldsFormatTestCase extends BaseIndexFileFormatTestCase {
 
   @Override
-  protected void addRandomFields(Document2 d) {
+  protected void addRandomFields(Document d) {
     FieldTypes fieldTypes = d.getFieldTypes();
     fieldTypes.setMultiValued("f");
     final int numValues = random().nextInt(3);
@@ -86,22 +76,18 @@ public abstract class BaseStoredFieldsFo
 
     final List<Integer> fieldIDs = new ArrayList<>();
 
-    FieldType customType = new FieldType(TextField.TYPE_STORED);
-    customType.setTokenized(false);
-    Field idField = newField("id", "", customType);
-
     for(int i=0;i<fieldCount;i++) {
       fieldIDs.add(i);
     }
 
-    final Map<String,Document2> docs = new HashMap<>();
+    final Map<String,Document> docs = new HashMap<>();
 
     if (VERBOSE) {
       System.out.println("TEST: build index docCount=" + docCount);
     }
 
     for(int i=0;i<docCount;i++) {
-      Document2 doc = w.newDocument();
+      Document doc = w.newDocument();
       final String id = ""+i;
       doc.addAtom("id", id);
       docs.put(id, doc);
@@ -155,8 +141,8 @@ public abstract class BaseStoredFieldsFo
           }
           TopDocs hits = s.search(new TermQuery(new Term("id", testID)), 1);
           assertEquals(1, hits.totalHits);
-          Document2 doc = r.document(hits.scoreDocs[0].doc);
-          Document2 docExp = docs.get(testID);
+          Document doc = r.document(hits.scoreDocs[0].doc);
+          Document docExp = docs.get(testID);
           for(int i=0;i<fieldCount;i++) {
             assertEquals("doc " + testID + ", field f" + fieldCount + " is wrong", docExp.get("f"+i),  doc.get("f"+i));
           }
@@ -173,7 +159,7 @@ public abstract class BaseStoredFieldsFo
   public void testStoredFieldsOrder() throws Throwable {
     Directory d = newDirectory();
     IndexWriter w = new IndexWriter(d, newIndexWriterConfig(new MockAnalyzer(random())));
-    Document2 doc = w.newDocument();
+    Document doc = w.newDocument();
     FieldTypes fieldTypes = w.getFieldTypes();
     fieldTypes.setMultiValued("zzz");
 
@@ -182,7 +168,7 @@ public abstract class BaseStoredFieldsFo
     doc.addStored("zzz", "1 2 3");
     w.addDocument(doc);
     IndexReader r = w.getReader();
-    Document2 doc2 = r.document(0);
+    Document doc2 = r.document(0);
     Iterator<IndexableField> it = doc2.iterator();
     assertTrue(it.hasNext());
     IndexableField f = it.next();
@@ -212,7 +198,7 @@ public abstract class BaseStoredFieldsFo
     for(int i=0;i<50;i++)
       b[i] = (byte) (i+77);
 
-    Document2 doc = w.newDocument();
+    Document doc = w.newDocument();
     doc.addBinary("binary", new BytesRef(b, 10, 17));
     BytesRef binaryValue = doc.getBinary("binary");
     assertEquals(10, binaryValue.offset);
@@ -221,7 +207,7 @@ public abstract class BaseStoredFieldsFo
     w.close();
 
     IndexReader ir = DirectoryReader.open(dir);
-    Document2 doc2 = ir.document(0);
+    Document doc2 = ir.document(0);
     IndexableField f2 = doc2.getField("binary");
     b = f2.binaryValue().bytes;
     assertTrue(b != null);
@@ -234,7 +220,7 @@ public abstract class BaseStoredFieldsFo
   public void testIndexedBit() throws Exception {
     Directory dir = newDirectory();
     RandomIndexWriter w = new RandomIndexWriter(random(), dir);
-    Document2 doc = w.newDocument();
+    Document doc = w.newDocument();
     doc.addStored("field", "value");
     doc.addAtom("field2", "value");
     w.addDocument(doc);
@@ -252,10 +238,6 @@ public abstract class BaseStoredFieldsFo
     iwConf.setMaxBufferedDocs(RandomInts.randomIntBetween(random(), 2, 30));
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwConf);
     
-    FieldType ft = new FieldType();
-    ft.setStored(true);
-    ft.freeze();
-
     final String string = TestUtil.randomSimpleString(random(), 50);
     final byte[] bytes = string.getBytes(StandardCharsets.UTF_8);
     final long l = random().nextBoolean() ? random().nextInt(42) : random().nextLong();
@@ -264,7 +246,7 @@ public abstract class BaseStoredFieldsFo
     final double d = random().nextDouble();
 
     for (int k = 0; k < 100; ++k) {
-      Document2 doc = iw.newDocument();
+      Document doc = iw.newDocument();
       doc.addStored("bytes", bytes);
       doc.addStored("string", string);
       doc.addInt("int", i);
@@ -278,7 +260,7 @@ public abstract class BaseStoredFieldsFo
     final DirectoryReader reader = DirectoryReader.open(dir);
     final int docID = random().nextInt(100);
     for (String fldName : new String[] {"bytes", "string", "int", "long", "float", "double"}) {
-      final Document2 sDoc = reader.document(docID, Collections.singleton(fldName));
+      final Document sDoc = reader.document(docID, Collections.singleton(fldName));
 
       final IndexableField sField = sDoc.getField(fldName);
       switch (fldName) {
@@ -316,7 +298,7 @@ public abstract class BaseStoredFieldsFo
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwConf);
     
     // make sure that the fact that documents might be empty is not a problem
-    final Document2 emptyDoc = iw.newDocument();
+    final Document emptyDoc = iw.newDocument();
     final int numDocs = random().nextBoolean() ? 1 : atLeast(1000);
     for (int i = 0; i < numDocs; ++i) {
       iw.addDocument(emptyDoc);
@@ -324,7 +306,7 @@ public abstract class BaseStoredFieldsFo
     iw.commit();
     final DirectoryReader rd = DirectoryReader.open(dir);
     for (int i = 0; i < numDocs; ++i) {
-      final Document2 doc = rd.document(i);
+      final Document doc = rd.document(i);
       assertNotNull(doc);
       assertTrue(doc.getFields().isEmpty());
     }
@@ -343,7 +325,7 @@ public abstract class BaseStoredFieldsFo
     // make sure the readers are properly cloned
     final int numDocs = atLeast(1000);
     for (int i = 0; i < numDocs; ++i) {
-      final Document2 doc = iw.newDocument();
+      final Document doc = iw.newDocument();
       doc.addAtom("fld", "" + i);
       iw.addDocument(doc);
     }
@@ -376,7 +358,7 @@ public abstract class BaseStoredFieldsFo
               if (topDocs.totalHits != 1) {
                 throw new IllegalStateException("Expected 1 hit, got " + topDocs.totalHits);
               }
-              final Document2 sdoc = rd.document(topDocs.scoreDocs[0].doc);
+              final Document sdoc = rd.document(topDocs.scoreDocs[0].doc);
               if (sdoc == null || sdoc.get("fld") == null) {
                 throw new IllegalStateException("Could not find document " + q);
               }
@@ -443,7 +425,7 @@ public abstract class BaseStoredFieldsFo
     }
 
     for (int i = 0; i < data.length; ++i) {
-      Document2 doc = iw.newDocument();
+      Document doc = iw.newDocument();
       doc.addInt("id", i);
       for (int j = 0; j < data[i].length; ++j) {
         doc.addStored("bytes" + j, new BytesRef(data[i][j]));
@@ -477,7 +459,7 @@ public abstract class BaseStoredFieldsFo
     assertTrue(ir.numDocs() > 0);
     int numDocs = 0;
     for (int i = 0; i < ir.maxDoc(); ++i) {
-      final Document2 doc = ir.document(i);
+      final Document doc = ir.document(i);
       if (doc == null) {
         continue;
       }
@@ -512,43 +494,31 @@ public abstract class BaseStoredFieldsFo
     IndexWriterConfig iwConf = newIndexWriterConfig(new MockAnalyzer(random()));
     iwConf.setMaxBufferedDocs(RandomInts.randomIntBetween(random(), 2, 30));
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwConf);
+    FieldTypes fieldTypes = iw.getFieldTypes();
+    fieldTypes.setMultiValued("fld");
 
     if (dir instanceof MockDirectoryWrapper) {
       ((MockDirectoryWrapper) dir).setThrottling(Throttling.NEVER);
     }
-
-    final Document emptyDoc = new Document(); // emptyDoc
-    final Document bigDoc1 = new Document(); // lot of small fields
-    final Document bigDoc2 = new Document(); // 1 very big field
-
-    final Field idField = new StringField("id", "", Store.NO);
-    emptyDoc.add(idField);
-    bigDoc1.add(idField);
-    bigDoc2.add(idField);
-
-    final FieldType onlyStored = new FieldType(StringField.TYPE_STORED);
-    onlyStored.setIndexOptions(IndexOptions.NONE);
-
-    final Field smallField = new Field("fld", randomByteArray(random().nextInt(10), 256), onlyStored);
     final int numFields = RandomInts.randomIntBetween(random(), 500000, 1000000);
-    for (int i = 0; i < numFields; ++i) {
-      bigDoc1.add(smallField);
-    }
-
-    final Field bigField = new Field("fld", randomByteArray(RandomInts.randomIntBetween(random(), 1000000, 5000000), 2), onlyStored);
-    bigDoc2.add(bigField);
-
     final int numDocs = atLeast(5);
-    final Document[] docs = new Document[numDocs];
-    for (int i = 0; i < numDocs; ++i) {
-      docs[i] = RandomPicks.randomFrom(random(), Arrays.asList(emptyDoc, bigDoc1, bigDoc2));
-    }
+    Document[] docs = new Document[numDocs];
     for (int i = 0; i < numDocs; ++i) {
-      idField.setStringValue("" + i);
-      iw.addDocument(docs[i]);
+      Document doc = iw.newDocument();
+      int x = random().nextInt(3);
+      doc.addAtom("id", "" + i);
+      if (x == 1) {
+        for (int j = 0; j < numFields; ++j) {
+          doc.addStored("fld", randomByteArray(random().nextInt(10), 256));
+        }
+      } else {
+        doc.addStored("fld", randomByteArray(RandomInts.randomIntBetween(random(), 1000000, 5000000), 2));
+      }
+      iw.addDocument(doc);
       if (random().nextInt(numDocs) == 0) {
         iw.commit();
       }
+      docs[i] = doc;
     }
     iw.commit();
     iw.forceMerge(1); // look at what happens when big docs are merged
@@ -558,12 +528,12 @@ public abstract class BaseStoredFieldsFo
       final Query query = new TermQuery(new Term("id", "" + i));
       final TopDocs topDocs = searcher.search(query, 1);
       assertEquals("" + i, 1, topDocs.totalHits);
-      final Document2 doc = rd.document(topDocs.scoreDocs[0].doc);
+      final Document doc = rd.document(topDocs.scoreDocs[0].doc);
       assertNotNull(doc);
       final List<IndexableField> fieldValues = doc.getFields("fld");
-      assertEquals(docs[i].getFields("fld").length, fieldValues.size());
+      assertEquals(docs[i].getFields("fld").size(), fieldValues.size());
       if (fieldValues.size() > 0) {
-        assertEquals(docs[i].getFields("fld")[0].binaryValue(), fieldValues.get(0).binaryValue());
+        assertEquals(docs[i].getFields("fld").get(0).binaryValue(), fieldValues.get(0).binaryValue());
       }
     }
     rd.close();
@@ -576,7 +546,7 @@ public abstract class BaseStoredFieldsFo
     Directory dir = newDirectory();
     RandomIndexWriter w = new RandomIndexWriter(random(), dir, newIndexWriterConfig(new MockAnalyzer(random())).setMergePolicy(NoMergePolicy.INSTANCE));
     for (int i = 0; i < numDocs; ++i) {
-      Document2 doc = w.newDocument();
+      Document doc = w.newDocument();
       doc.addAtom("id", Integer.toString(i));
       doc.addAtom("f", TestUtil.randomSimpleString(random()));
       w.addDocument(doc);

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java?rev=1642535&r1=1642534&r2=1642535&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java Sun Nov 30 11:07:09 2014
@@ -35,14 +35,8 @@ import org.apache.lucene.analysis.tokena
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.TermVectorsFormat;
-import org.apache.lucene.document.Document2;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field.Store;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.LowSchemaField;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.TermsEnum.SeekStatus;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.TermQuery;
@@ -99,11 +93,12 @@ public abstract class BaseTermVectorsFor
   }
 
   @Override
-  protected void addRandomFields(Document2 doc) {
+  protected void addRandomFields(Document doc) {
     for (Options opts : validOptions()) {
       final int numFields = random().nextInt(5);
       for (int j = 0; j < numFields; ++j) {
-        LowSchemaField field = new LowSchemaField("f_" + opts, TestUtil.randomSimpleString(random(), 2), IndexOptions.DOCS_AND_FREQS_AND_POSITIONS, true);
+        LowSchemaField field = new LowSchemaField(doc.getFieldTypes().getIndexAnalyzer(),
+                                                  "f_" + opts, TestUtil.randomSimpleString(random(), 2), IndexOptions.DOCS_AND_FREQS_AND_POSITIONS, true);
         field.enableTermVectors(opts.positions, opts.offsets, opts.payloads);
         doc.add(field);
       }
@@ -307,10 +302,11 @@ public abstract class BaseTermVectorsFor
       }
     }
 
-    public Document2 toDocument(IndexWriter w) {
-      final Document2 doc = w.newDocument();
+    public Document toDocument(IndexWriter w) {
+      final Document doc = w.newDocument();
       for (int i = 0; i < fieldNames.length; ++i) {
-        LowSchemaField field = new LowSchemaField(fieldNames[i], null, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS, true);
+        LowSchemaField field = new LowSchemaField(doc.getFieldTypes().getIndexAnalyzer(),
+                                                  fieldNames[i], null, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS, true);
         field.doNotStore();
         field.setTokenStream(tokenStreams[i]);
         field.enableTermVectors(options.positions, options.offsets, options.payloads);
@@ -494,7 +490,7 @@ public abstract class BaseTermVectorsFor
     }
   }
 
-  protected Document2 addId(Document2 doc, String id) {
+  protected Document addId(Document doc, String id) {
     doc.addAtom("id", id);
     return doc;
   }
@@ -511,7 +507,7 @@ public abstract class BaseTermVectorsFor
       final int docWithVectors = random().nextInt(numDocs);
       final Directory dir = newDirectory();
       final RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
-      final Document2 emptyDoc = writer.newDocument();
+      final Document emptyDoc = writer.newDocument();
       final RandomDocument doc = docFactory.newDocument(TestUtil.nextInt(random(), 1, 3), 20, options);
       for (int i = 0; i < numDocs; ++i) {
         if (i == docWithVectors) {

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/DocHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/DocHelper.java?rev=1642535&r1=1642534&r2=1642535&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/DocHelper.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/DocHelper.java Sun Nov 30 11:07:09 2014
@@ -30,14 +30,8 @@ import java.util.Set;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenizer;
-import org.apache.lucene.document.Document2;
 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.StoredField;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.document.TextField;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.store.Directory;
@@ -126,7 +120,7 @@ class DocHelper {
    * Adds the fields above to a document 
    * @param doc The document to write
    */ 
-  private static void setupDoc(FieldTypes fieldTypes, Document2 doc) {
+  private static void setupDoc(FieldTypes fieldTypes, Document doc) {
 
     fieldTypes.enableTermVectors(TEXT_FIELD_2_KEY);
     fieldTypes.disableHighlighting(TEXT_FIELD_2_KEY);
@@ -256,7 +250,7 @@ class DocHelper {
   public static SegmentCommitInfo writeDoc(Random random, Directory dir, Analyzer analyzer, Similarity similarity) throws IOException {
     IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig( /* LuceneTestCase.newIndexWriterConfig(random, */ 
         analyzer).setSimilarity(similarity == null ? IndexSearcher.getDefaultSimilarity() : similarity));
-    Document2 doc = writer.newDocument();
+    Document doc = writer.newDocument();
     setupDoc(writer.getFieldTypes(), doc);
     writer.addDocument(doc);
     writer.commit();
@@ -269,11 +263,11 @@ class DocHelper {
     return 14;
   }
 
-  public static int numFields(Document2 doc) {
+  public static int numFields(Document doc) {
     return doc.getFields().size();
   }
   
-  public static Document2 createDocument(IndexWriter writer, int n, String indexName, int numFields) {
+  public static Document createDocument(IndexWriter writer, int n, String indexName, int numFields) {
     StringBuilder sb = new StringBuilder();
 
     FieldTypes fieldTypes = writer.getFieldTypes();
@@ -286,7 +280,7 @@ class DocHelper {
     fieldTypes.enableTermVectorPositions("id");
     fieldTypes.enableTermVectorOffsets("id");
 
-    Document2 doc = writer.newDocument();
+    Document doc = writer.newDocument();
     doc.addAtom("id", Integer.toString(n));
     doc.addAtom("indexname", indexName);
     sb.append("a");

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java?rev=1642535&r1=1642534&r2=1642535&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java Sun Nov 30 11:07:09 2014
@@ -25,8 +25,7 @@ import java.util.Random;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.document.Document2;
-import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Document;
 import org.apache.lucene.document.FieldTypes;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ReferenceManager;
@@ -102,7 +101,7 @@ public class RandomIndexWriter implement
     return w.getFieldTypes();
   }
 
-  public Document2 newDocument() {
+  public Document newDocument() {
     return w.newDocument();
   }
 

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java?rev=1642535&r1=1642534&r2=1642535&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java Sun Nov 30 11:07:09 2014
@@ -27,9 +27,7 @@ import java.util.concurrent.atomic.Atomi
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document2;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.PhraseQuery;
 import org.apache.lucene.search.Query;
@@ -149,7 +147,7 @@ public abstract class ThreadedIndexingAn
                   }
                 }
 
-                Document2 doc = docs.nextDoc();
+                Document doc = docs.nextDoc();
                 
                 // Maybe add randomly named field
                 if (random().nextBoolean()) {
@@ -176,7 +174,7 @@ public abstract class ThreadedIndexingAn
 
                     final List<String> docIDs = new ArrayList<>();
                     final SubDocs subDocs = new SubDocs(packID, docIDs);
-                    final List<Document2> docsList = new ArrayList<>();
+                    final List<Document> docsList = new ArrayList<>();
 
                     allSubDocs.add(subDocs);
                     doc.addAtom("packID", packID);
@@ -459,7 +457,7 @@ public abstract class ThreadedIndexingAn
         final int inc = Math.max(1, maxDoc/50);
         for(int docID=0;docID<maxDoc;docID += inc) {
           if (liveDocs == null || liveDocs.get(docID)) {
-            final Document2 doc = reader.document(docID);
+            final Document doc = reader.document(docID);
             sum += doc.getFields().size();
           }
         }
@@ -571,7 +569,7 @@ public abstract class ThreadedIndexingAn
               startDocID = docID;
             }
             lastDocID = docID;
-            final Document2 doc = s.doc(docID);
+            final Document doc = s.doc(docID);
             assertEquals(subDocs.packID, doc.get("packID"));
           }
 

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/search/BaseExplanationTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/search/BaseExplanationTestCase.java?rev=1642535&r1=1642534&r2=1642535&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/search/BaseExplanationTestCase.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/search/BaseExplanationTestCase.java Sun Nov 30 11:07:09 2014
@@ -18,10 +18,7 @@ package org.apache.lucene.search;
  */
 
 import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document2;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
@@ -32,7 +29,6 @@ import org.apache.lucene.search.spans.Sp
 import org.apache.lucene.search.spans.SpanQuery;
 import org.apache.lucene.search.spans.SpanTermQuery;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -74,7 +70,7 @@ public abstract class BaseExplanationTes
     directory = newDirectory();
     RandomIndexWriter writer = new RandomIndexWriter(random(), directory, newIndexWriterConfig(new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy()));
     for (int i = 0; i < docFields.length; i++) {
-      Document2 doc = writer.newDocument();
+      Document doc = writer.newDocument();
       doc.addAtom(KEY, ""+i);
       doc.addLargeText(FIELD, docFields[i], (float) i);
       doc.addLargeText(ALTFIELD, docFields[i]);

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java?rev=1642535&r1=1642534&r2=1642535&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java Sun Nov 30 11:07:09 2014
@@ -24,7 +24,6 @@ import java.util.Random;
 import junit.framework.Assert;
 
 import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document;
 import org.apache.lucene.index.AllDeletedFilterReader;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java?rev=1642535&r1=1642534&r2=1642535&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java Sun Nov 30 11:07:09 2014
@@ -23,11 +23,7 @@ import java.util.Random;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenizer;
-import org.apache.lucene.document.Document2;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
@@ -65,7 +61,7 @@ public abstract class SearchEquivalenceT
     // index some docs
     int numDocs = atLeast(1000);
     for (int i = 0; i < numDocs; i++) {
-      Document2 doc = iw.newDocument();
+      Document doc = iw.newDocument();
       doc.addUniqueAtom("id", Integer.toString(i));
       doc.addLargeText("field", randomFieldContents());
       iw.addDocument(doc);

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocs.java?rev=1642535&r1=1642534&r2=1642535&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocs.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocs.java Sun Nov 30 11:07:09 2014
@@ -34,16 +34,8 @@ import java.util.Random;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.zip.GZIPInputStream;
 
-import org.apache.lucene.document.Document2;
 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.StringField;
-import org.apache.lucene.document.TextField;
-import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.IndexWriter;
 
 /** Minimal port of benchmark's LneDocSource +
@@ -165,7 +157,7 @@ public class LineFileDocs implements Clo
   private final static char SEP = '\t';
 
   /** Note: Document instance is re-used per-thread */
-  public Document2 nextDoc() throws IOException {
+  public Document nextDoc() throws IOException {
     String line;
     synchronized(this) {
       line = reader.readLine();
@@ -189,7 +181,7 @@ public class LineFileDocs implements Clo
       throw new RuntimeException("line: [" + line + "] is in an invalid format !");
     }
 
-    Document2 doc = w.newDocument();
+    Document doc = w.newDocument();
     doc.addLargeText("body", line.substring(1+spot2, line.length()));
 
     final String title = line.substring(0, spot);

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocsText.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocsText.java?rev=1642535&r1=1642534&r2=1642535&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocsText.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocsText.java Sun Nov 30 11:07:09 2014
@@ -34,17 +34,6 @@ import java.util.Random;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.zip.GZIPInputStream;
 
-import org.apache.lucene.document.Document2;
-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.SortedDocValuesField;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.document.TextField;
-import org.apache.lucene.index.IndexOptions;
-import org.apache.lucene.index.IndexWriter;
-
 /** Just provides the parsed body, title, date, id from the line file docs. */
 public class LineFileDocsText implements Closeable {
 

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java?rev=1642535&r1=1642534&r2=1642535&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java Sun Nov 30 11:07:09 2014
@@ -57,12 +57,7 @@ import java.util.logging.Logger;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document2;
-import org.apache.lucene.document.Field.Store;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.FieldType;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.document.TextField;
+import org.apache.lucene.document.Document;
 import org.apache.lucene.index.AlcoholicMergePolicy;
 import org.apache.lucene.index.AssertingDirectoryReader;
 import org.apache.lucene.index.AssertingLeafReader;
@@ -77,7 +72,6 @@ import org.apache.lucene.index.FieldFilt
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.Fields;
-import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.IndexReader.ReaderClosedListener;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
@@ -643,8 +637,6 @@ public abstract class LuceneTestCase ext
     .around(new TestRuleSetupAndRestoreInstanceEnv())
     .around(parentChainCallRule);
 
-  private static final Map<String,FieldType> fieldToType = new HashMap<String,FieldType>();
-
   enum LiveIWCFlushMode {BY_RAM, BY_DOCS, EITHER};
 
   /** Set by TestRuleSetupAndRestoreClassEnv */
@@ -672,7 +664,6 @@ public abstract class LuceneTestCase ext
   @After
   public void tearDown() throws Exception {
     parentChainCallRule.teardownCalled = true;
-    fieldToType.clear();
 
     // Test is supposed to call this itself, but we do this defensively in case it forgot:
     restoreIndexWriterMaxDocs();
@@ -895,6 +886,11 @@ public abstract class LuceneTestCase ext
     return new RandomIndexWriter(random(), dir, newIndexWriterConfig());
   }
 
+  /** create a new index writer config with random defaults, using MockAnalyzer */
+  public static RandomIndexWriter newRandomIndexWriter(Directory dir, Analyzer a) throws IOException {
+    return new RandomIndexWriter(random(), dir, newIndexWriterConfig(a));
+  }
+
   /** create a new index writer config with random defaults */
   public static IndexWriterConfig newIndexWriterConfig(Analyzer a) {
     return newIndexWriterConfig(random(), a);
@@ -1379,112 +1375,7 @@ public abstract class LuceneTestCase ext
     }
   }
   
-  public static Field newStringField(String name, String value, Store stored) {
-    return newField(random(), name, value, stored == Store.YES ? StringField.TYPE_STORED : StringField.TYPE_NOT_STORED);
-  }
-
-  public static Field newTextField(String name, String value, Store stored) {
-    return newField(random(), name, value, stored == Store.YES ? TextField.TYPE_STORED : TextField.TYPE_NOT_STORED);
-  }
-  
-  public static Field newStringField(Random random, String name, String value, Store stored) {
-    return newField(random, name, value, stored == Store.YES ? StringField.TYPE_STORED : StringField.TYPE_NOT_STORED);
-  }
-  
-  public static Field newTextField(Random random, String name, String value, Store stored) {
-    return newField(random, name, value, stored == Store.YES ? TextField.TYPE_STORED : TextField.TYPE_NOT_STORED);
-  }
-  
-  public static Field newField(String name, String value, FieldType type) {
-    return newField(random(), name, value, type);
-  }
-
-  /** Returns a FieldType derived from newType but whose
-   *  term vector options match the old type */
-  private static FieldType mergeTermVectorOptions(FieldType newType, FieldType oldType) {
-    if (newType.indexOptions() != IndexOptions.NONE && oldType.storeTermVectors() == true && newType.storeTermVectors() == false) {
-      newType = new FieldType(newType);
-      newType.setStoreTermVectors(oldType.storeTermVectors());
-      newType.setStoreTermVectorPositions(oldType.storeTermVectorPositions());
-      newType.setStoreTermVectorOffsets(oldType.storeTermVectorOffsets());
-      newType.setStoreTermVectorPayloads(oldType.storeTermVectorPayloads());
-      newType.freeze();
-    }
-
-    return newType;
-  }
-
-  // nocommit can we use FieldTypes here?
-
-  // TODO: if we can pull out the "make term vector options
-  // consistent across all instances of the same field name"
-  // write-once schema sort of helper class then we can
-  // remove the sync here.  We can also fold the random
-  // "enable norms" (now commented out, below) into that:
-  public synchronized static Field newField(Random random, String name, String value, FieldType type) {
-
-    // Defeat any consumers that illegally rely on intern'd
-    // strings (we removed this from Lucene a while back):
-    name = new String(name);
-
-    FieldType prevType = fieldToType.get(name);
-
-    if (usually(random) || type.indexOptions() == IndexOptions.NONE || prevType != null) {
-      // most of the time, don't modify the params
-      if (prevType == null) {
-        fieldToType.put(name, new FieldType(type));
-      } else {
-        type = mergeTermVectorOptions(type, prevType);
-      }
-
-      return new Field(name, value, type);
-    }
-
-    // TODO: once all core & test codecs can index
-    // offsets, sometimes randomly turn on offsets if we are
-    // already indexing positions...
-
-    FieldType newType = new FieldType(type);
-    if (!newType.stored() && random.nextBoolean()) {
-      newType.setStored(true); // randomly store it
-    }
-
-    // Randomly turn on term vector options, but always do
-    // so consistently for the same field name:
-    if (!newType.storeTermVectors() && random.nextBoolean()) {
-      newType.setStoreTermVectors(true);
-      if (!newType.storeTermVectorPositions()) {
-        newType.setStoreTermVectorPositions(random.nextBoolean());
-        
-        if (newType.storeTermVectorPositions()) {
-          if (!newType.storeTermVectorPayloads()) {
-            newType.setStoreTermVectorPayloads(random.nextBoolean());
-          }
-        }
-      }
-      
-      if (!newType.storeTermVectorOffsets()) {
-        newType.setStoreTermVectorOffsets(random.nextBoolean());
-      }
-
-      if (VERBOSE) {
-        System.out.println("NOTE: LuceneTestCase: upgrade name=" + name + " type=" + newType);
-      }
-    }
-    newType.freeze();
-    fieldToType.put(name, newType);
-
-    // TODO: we need to do this, but smarter, ie, most of
-    // the time we set the same value for a given field but
-    // sometimes (rarely) we change it up:
-    /*
-    if (newType.omitNorms()) {
-      newType.setOmitNorms(random.nextBoolean());
-    }
-    */
-    
-    return new Field(name, value, newType);
-  }
+  // nocommit how to randomize FieldTypes here?
 
   /** 
    * Return a random Locale from the available locales on the system.
@@ -2243,8 +2134,8 @@ public abstract class LuceneTestCase ext
   public void assertStoredFieldsEquals(String info, IndexReader leftReader, IndexReader rightReader) throws IOException {
     assert leftReader.maxDoc() == rightReader.maxDoc();
     for (int i = 0; i < leftReader.maxDoc(); i++) {
-      Document2 leftDoc = leftReader.document(i);
-      Document2 rightDoc = rightReader.document(i);
+      Document leftDoc = leftReader.document(i);
+      Document rightDoc = rightReader.document(i);
       
       // TODO: I think this is bogus because we don't document what the order should be
       // from these iterators, etc. I think the codec/IndexReader should be free to order this stuff

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java?rev=1642535&r1=1642534&r2=1642535&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java Sun Nov 30 11:07:09 2014
@@ -54,12 +54,6 @@ import org.apache.lucene.codecs.lucene50
 import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat;
 import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
-import org.apache.lucene.document.BinaryDocValuesField;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.FieldType.NumericType;
-import org.apache.lucene.document.NumericDocValuesField;
-import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.index.CheckIndex;
 import org.apache.lucene.index.ConcurrentMergeScheduler;
 import org.apache.lucene.index.DocValuesType;