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/04 10:31:25 UTC

svn commit: r1636528 - in /lucene/dev/branches/lucene6005/lucene/core/src: java/org/apache/lucene/codecs/blocktree/ java/org/apache/lucene/document/ java/org/apache/lucene/index/ test/org/apache/lucene/codecs/compressing/ test/org/apache/lucene/codecs/...

Author: mikemccand
Date: Tue Nov  4 09:31:24 2014
New Revision: 1636528

URL: http://svn.apache.org/r1636528
Log:
LUCENE-6005: fix sneaky auto-prefix bug, cutover more tests

Modified:
    lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java
    lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
    lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java
    lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java
    lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/Document.java
    lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/Document2.java
    lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/FieldTypes.java
    lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java
    lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingTermVectorsFormat.java
    lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestAutoPrefixTerms.java
    lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestBlockPostingsFormat.java
    lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestBlockPostingsFormat2.java
    lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/document/TestDocument2.java
    lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java
    lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestPrefixQuery.java
    lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestTermRangeQuery.java
    lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestWildcard.java
    lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestBufferedIndexInput.java
    lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestLockFactory.java
    lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java
    lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestRAMDirectory.java
    lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestWindowsMMap.java
    lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java?rev=1636528&r1=1636527&r2=1636528&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java Tue Nov  4 09:31:24 2014
@@ -45,8 +45,8 @@ import org.apache.lucene.util.StringHelp
 class AutoPrefixTermsWriter {
 
   //static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
-  //static boolean DEBUG = false;
   //static boolean DEBUG2 = BlockTreeTermsWriter.DEBUG2;
+  //static boolean DEBUG = true;
   //static boolean DEBUG2 = true;
 
   /** Describes a range of term-space to match, either a simple prefix
@@ -80,7 +80,6 @@ class AutoPrefixTermsWriter {
       assert prefix.length > 0 || floorLeadStart != -2 || floorLeadEnd != 0xff;
     }
 
-    /*
     @Override
     public String toString() {
       String s = brToString(new BytesRef(prefix));
@@ -91,7 +90,6 @@ class AutoPrefixTermsWriter {
       }
       return s;
     }
-    */
 
     @Override
     public int compareTo(PrefixTerm other) {
@@ -157,10 +155,9 @@ class AutoPrefixTermsWriter {
   }
 
   // for debugging
-  /*
   static String brToString(BytesRef b) {
     try {
-      return b.utf8ToString() + " " + b;
+      return b.utf8ToString() + " " + b + " len=" + b.length;
     } catch (Throwable t) {
       // If BytesRef isn't actually UTF8, or it's eg a
       // prefix of UTF8 that ends mid-unicode-char, we
@@ -168,7 +165,6 @@ class AutoPrefixTermsWriter {
       return b.toString();
     }
   }
-  */
 
   final List<PrefixTerm> prefixes = new ArrayList<>();
   private final int minItemsInPrefix;
@@ -183,13 +179,13 @@ class AutoPrefixTermsWriter {
   private int[] prefixStarts = new int[8];
   private List<Object> pending = new ArrayList<>();
 
-  //private final String segment;
+  private final String segment;
 
-  public AutoPrefixTermsWriter(Terms terms, int minItemsInPrefix, int maxItemsInPrefix) throws IOException {
+  public AutoPrefixTermsWriter(String segment, Terms terms, int minItemsInPrefix, int maxItemsInPrefix) throws IOException {
     this.minItemsInPrefix = minItemsInPrefix;
     this.maxItemsInPrefix = maxItemsInPrefix;
-    //this.segment = segment;
-
+    this.segment = segment;
+    //if (DEBUG) System.out.println("autoprefix terms=" + terms);
     TermsEnum termsEnum = terms.iterator(null);
     while (true) {
       BytesRef term = termsEnum.next();
@@ -199,6 +195,7 @@ class AutoPrefixTermsWriter {
       //if (DEBUG) System.out.println("pushTerm: " + brToString(term));
       pushTerm(term);
     }
+    //if (DEBUG) System.out.println("done push terms");
 
     if (pending.size() > 1) {
       pushTerm(BlockTreeTermsWriter.EMPTY_BYTES_REF);
@@ -306,7 +303,7 @@ class AutoPrefixTermsWriter {
       }
       pendingCount++;
 
-      //if (DEBUG) System.out.println("    check term=" + brToString(new BytesRef(termBytes)));
+      //if (DEBUG) System.out.println("    check term=" + brToString(new BytesRef(termBytes)) + " pt=" + ptEntry);
 
       int suffixLeadLabel;
 
@@ -413,7 +410,7 @@ class AutoPrefixTermsWriter {
     assert floorLeadEnd != -1;
 
     PrefixTerm pt = new PrefixTerm(prefix, floorLeadStart, floorLeadEnd); 
-    //if (DEBUG2) System.out.println("    savePrefix: seg=" + segment + " " + pt + " count=" + count);
+    //if (DEBUG2) System.out.println("    savePrefix: seg=" + segment + " " + pt);
     prefixes.add(pt);
   }
 }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java?rev=1636528&r1=1636527&r2=1636528&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java Tue Nov  4 09:31:24 2014
@@ -26,13 +26,12 @@ import org.apache.lucene.codecs.CodecUti
 import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.PostingsWriterBase;
 import org.apache.lucene.codecs.blocktree.AutoPrefixTermsWriter.PrefixTerm;
-import org.apache.lucene.document.FieldType;
 import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
@@ -213,8 +212,8 @@ public final class BlockTreeTermsWriter 
    *  #BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int)}. */
   public final static int DEFAULT_MAX_BLOCK_SIZE = 48;
 
-  //public static boolean DEBUG = false;
-  //public static boolean DEBUG2 = false;
+  //public static boolean DEBUG = true;
+  //public static boolean DEBUG2 = true;
 
   //private final static boolean SAVE_DOT_FILES = false;
 
@@ -260,7 +259,7 @@ public final class BlockTreeTermsWriter 
 
   private final List<FieldMetaData> fields = new ArrayList<>();
 
-  // private final String segment;
+  private final String segment;
 
   final FixedBitSet prefixDocs;
 
@@ -310,6 +309,7 @@ public final class BlockTreeTermsWriter 
 
     this.maxDoc = state.segmentInfo.getDocCount();
     this.fieldInfos = state.fieldInfos;
+    this.segment = state.segmentInfo.name;
 
     if (minItemsInAutoPrefix != 0) {
       // TODO: can we used compressed bitset instead?  that auto-upgrades if it's dense enough...
@@ -412,10 +412,12 @@ public final class BlockTreeTermsWriter 
       // First pass to find all prefix terms we should compile into the index:
       List<PrefixTerm> prefixTerms;
       if (minItemsInAutoPrefix != 0) {
+        //if (DEBUG) System.out.println("now write auto-prefix terms field=" + fieldInfo.name);
+
         if (fieldInfo.getIndexOptions() != IndexOptions.DOCS) {
           throw new IllegalStateException("ranges can only be indexed with IndexOptions.DOCS (field: " + fieldInfo.name + ")");
         }
-        prefixTerms = new AutoPrefixTermsWriter(terms, minItemsInAutoPrefix, maxItemsInAutoPrefix).prefixes;
+        prefixTerms = new AutoPrefixTermsWriter(segment, terms, minItemsInAutoPrefix, maxItemsInAutoPrefix).prefixes;
         //if (DEBUG) {
         //  for(PrefixTerm term : prefixTerms) {
         //    System.out.println("field=" + fieldInfo.name + " PREFIX TERM: " + term);
@@ -795,7 +797,7 @@ public final class BlockTreeTermsWriter 
       System.arraycopy(lastTerm.get().bytes, 0, prefix.bytes, 0, prefixLength);
       prefix.length = prefixLength;
 
-      //if (DEBUG2) System.out.println("    writeBlock field=" + fieldInfo.name + " prefix=" + brToString(prefix) + " fp=" + startFP + " isFloor=" + isFloor + " isLastInFloor=" + (end == pending.size()) + " floorLeadLabel=" + floorLeadLabel + " start=" + start + " end=" + end + " hasTerms=" + hasTerms + " hasSubBlocks=" + hasSubBlocks);
+      //if (DEBUG2) System.out.println("  writeBlock seg=" + segment + " prefixLength=" + prefixLength + " entCount=" + (end-start) + " field=" + fieldInfo.name + " prefix=" + brToString(prefix) + " fp=" + startFP + " isFloor=" + isFloor + " isLastInFloor=" + (end == pending.size()) + " floorLeadLabel=" + floorLeadLabel + " start=" + start + " end=" + end + " hasTerms=" + hasTerms + " hasSubBlocks=" + hasSubBlocks + " hasPrefixTerms=" + hasPrefixTerms + " pending.size()=" + pending.size());
 
       // Write block header:
       int numEntries = end - start;
@@ -806,12 +808,6 @@ public final class BlockTreeTermsWriter 
       }
       termsOut.writeVInt(code);
 
-      /*
-      if (DEBUG) {
-        System.out.println("  writeBlock " + (isFloor ? "(floor) " : "") + "seg=" + segment + " pending.size()=" + pending.size() + " prefixLength=" + prefixLength + " indexPrefix=" + brToString(prefix) + " entCount=" + (end-start+1) + " startFP=" + startFP + (isFloor ? (" floorLeadLabel=" + Integer.toHexString(floorLeadLabel)) : ""));
-      }
-      */
-
       // 1st pass: pack term suffix bytes into byte[] blob
       // TODO: cutover to bulk int codec... simple64?
 
@@ -908,6 +904,7 @@ public final class BlockTreeTermsWriter 
                 code |= 3;
               }
             }
+            //if (DEBUG) System.out.println("    write suffix @ pos=" + suffixWriter.getFilePointer());
             suffixWriter.writeVInt(code);
             suffixWriter.writeBytes(term.termBytes, prefixLength, suffix);
             if (floorLeadEnd != -1) {

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java?rev=1636528&r1=1636527&r2=1636528&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java Tue Nov  4 09:31:24 2014
@@ -52,7 +52,7 @@ final class SegmentTermsEnum extends Ter
 
   private int targetBeforeCurrentLength;
 
-  //static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
+  // static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
 
   private final ByteArrayDataInput scratchReader = new ByteArrayDataInput();
 
@@ -258,6 +258,7 @@ final class SegmentTermsEnum extends Ter
   // Pushes next'd frame or seek'd frame; we later
   // lazy-load the frame only when needed
   SegmentTermsEnumFrame pushFrame(FST.Arc<BytesRef> arc, long fp, int length) throws IOException {
+    //if (DEBUG) System.out.println("pushFrame length=" + length + " fp=" + fp);
     final SegmentTermsEnumFrame f = getFrame(1+currentFrame.ord);
     f.arc = arc;
     if (f.fpOrig == fp && f.nextEnt != -1) {
@@ -300,7 +301,6 @@ final class SegmentTermsEnum extends Ter
     return true;
   }
 
-  /*
   // for debugging
   @SuppressWarnings("unused")
   static String brToString(BytesRef b) {
@@ -319,7 +319,6 @@ final class SegmentTermsEnum extends Ter
   static String brToString(BytesRefBuilder b) {
     return brToString(b.get());
   }
-  */
 
   @Override
   public boolean seekExact(BytesRef target) throws IOException {
@@ -902,10 +901,10 @@ final class SegmentTermsEnum extends Ter
     targetBeforeCurrentLength = currentFrame.ord;
 
     assert !eof;
-    // if (DEBUG) {
-    //   System.out.println("\nBTTR.next seg=" + fr.parent.segment + " term=" + brToString(term) + " termExists?=" + termExists + " field=" + fr.fieldInfo.name + " termBlockOrd=" + currentFrame.state.termBlockOrd + " validIndexPrefix=" + validIndexPrefix);
-    //   printSeekState(System.out);
-    // }
+    //if (DEBUG) {
+    //  System.out.println("\nBTTR.next seg=" + fr.parent.segment + " term=" + brToString(term) + " termExists?=" + termExists + " field=" + fr.fieldInfo.name + " termBlockOrd=" + currentFrame.state.termBlockOrd + " validIndexPrefix=" + validIndexPrefix);
+    //  printSeekState(System.out);
+    //}
 
     if (currentFrame == staticFrame) {
       // If seek was previously called and the term was
@@ -914,7 +913,7 @@ final class SegmentTermsEnum extends Ter
       // docFreq, etc.  But, if they then call next(),
       // this method catches up all internal state so next()
       // works properly:
-      //if (DEBUG) System.out.println("  re-seek to pending term=" + term.utf8ToString() + " " + term);
+      //if (DEBUG) System.out.println("  re-seek to pending term=" + brToString(term) + " " + term);
       final boolean result = seekExact(term.get());
       assert result;
     }
@@ -950,9 +949,7 @@ final class SegmentTermsEnum extends Ter
         // Note that the seek state (last seek) has been
         // invalidated beyond this depth
         validIndexPrefix = Math.min(validIndexPrefix, currentFrame.prefix);
-        //if (DEBUG) {
-        //System.out.println("  reset validIndexPrefix=" + validIndexPrefix);
-        //}
+        //if (DEBUG) System.out.println("  reset validIndexPrefix=" + validIndexPrefix);
       }
     }
 

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java?rev=1636528&r1=1636527&r2=1636528&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java Tue Nov  4 09:31:24 2014
@@ -101,6 +101,7 @@ final class SegmentTermsEnumFrame {
     this.state.totalTermFreq = -1;
     this.longs = new long[ste.fr.longsSize];
     this.versionAutoPrefix = ste.fr.parent.version >= BlockTreeTermsReader.VERSION_AUTO_PREFIX_TERMS;
+    //System.out.println("STE.init seg=" + ste.fr.parent.segment + " versionAutoPrefix=" + versionAutoPrefix);
   }
 
   public void setFloorData(ByteArrayDataInput in, BytesRef source) {
@@ -122,9 +123,7 @@ final class SegmentTermsEnumFrame {
   }
 
   void loadNextFloorBlock() throws IOException {
-    //if (DEBUG) {
-    //System.out.println("    loadNextFloorBlock fp=" + fp + " fpEnd=" + fpEnd);
-    //}
+    //if (DEBUG) System.out.println("    loadNextFloorBlock fp=" + fp + " fpEnd=" + fpEnd);
     assert arc == null || isFloor: "arc=" + arc + " isFloor=" + isFloor;
     fp = fpEnd;
     nextEnt = -1;
@@ -142,6 +141,8 @@ final class SegmentTermsEnumFrame {
      use. */
   void loadBlock() throws IOException {
 
+    //if (DEBUG) System.out.println("loadBlock fp=" + fp);
+
     // Clone the IndexInput lazily, so that consumers
     // that just pull a TermsEnum to
     // seekExact(TermState) don't pay this cost:
@@ -169,6 +170,7 @@ final class SegmentTermsEnumFrame {
     // term suffixes:
     code = ste.in.readVInt();
     isLeafBlock = (code & 1) != 0;
+    //if (DEBUG) System.out.println("  isLeafBlock=" + isLeafBlock);
     int numBytes = code >>> 1;
     if (suffixBytes.length < numBytes) {
       suffixBytes = new byte[ArrayUtil.oversize(numBytes, 1)];
@@ -176,13 +178,13 @@ final class SegmentTermsEnumFrame {
     ste.in.readBytes(suffixBytes, 0, numBytes);
     suffixesReader.reset(suffixBytes, 0, numBytes);
 
-    /*if (DEBUG) {
-      if (arc == null) {
-      System.out.println("    loadBlock (next) fp=" + fp + " entCount=" + entCount + " prefixLen=" + prefix + " isLastInFloor=" + isLastInFloor + " leaf?=" + isLeafBlock);
-      } else {
-      System.out.println("    loadBlock (seek) fp=" + fp + " entCount=" + entCount + " prefixLen=" + prefix + " hasTerms?=" + hasTerms + " isFloor?=" + isFloor + " isLastInFloor=" + isLastInFloor + " leaf?=" + isLeafBlock);
-      }
-      }*/
+    //if (DEBUG) {
+    //  if (arc == null) {
+    //    System.out.println("    loadBlock (next) fp=" + fp + " entCount=" + entCount + " prefixLen=" + prefix + " isLastInFloor=" + isLastInFloor + " leaf?=" + isLeafBlock);
+    //  } else {
+    //    System.out.println("    loadBlock (seek) fp=" + fp + " entCount=" + entCount + " prefixLen=" + prefix + " hasTerms?=" + hasTerms + " isFloor?=" + isFloor + " isLastInFloor=" + isLastInFloor + " leaf?=" + isLeafBlock);
+    //  }
+    //}
 
     // stats
     numBytes = ste.in.readVInt();
@@ -210,7 +212,6 @@ final class SegmentTermsEnumFrame {
     ste.in.readBytes(bytes, 0, numBytes);
     bytesReader.reset(bytes, 0, numBytes);
 
-
     // Sub-blocks of a single floor block are always
     // written one after another -- tail recurse:
     fpEnd = ste.in.getFilePointer();
@@ -278,10 +279,11 @@ final class SegmentTermsEnumFrame {
   }
 
   public void nextLeaf() {
-    //if (DEBUG) System.out.println("  frame.next ord=" + ord + " nextEnt=" + nextEnt + " entCount=" + entCount);
+    //if (DEBUG) System.out.println("  frame.next ord=" + ord + " nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp);
     assert nextEnt != -1 && nextEnt < entCount: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
     nextEnt++;
     suffix = suffixesReader.readVInt();
+    //if (DEBUG) System.out.println("    suffix=" + suffix + " prefix=" + prefix);
     startBytePos = suffixesReader.getPosition();
     ste.term.setLength(prefix + suffix);
     ste.term.grow(ste.term.length());
@@ -294,18 +296,26 @@ final class SegmentTermsEnumFrame {
     while (true) {
       if (nextEnt == entCount) {
         assert arc == null || (isFloor && isLastInFloor == false): "isFloor=" + isFloor + " isLastInFloor=" + isLastInFloor;
+        //if (DEBUG) System.out.println("    stef: loadNextFloorBlock");
         loadNextFloorBlock();
-        continue;
+        if (isLeafBlock) {
+          nextLeaf();
+          return false;
+        } else {
+          continue;
+        }
       }
         
       assert nextEnt != -1 && nextEnt < entCount: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
       nextEnt++;
+      //System.out.println("    stef: readSuffix @ fp=" + suffixesReader.getPosition());
       final int code = suffixesReader.readVInt();
       if (versionAutoPrefix == false) {
         suffix = code >>> 1;
       } else {
         suffix = code >>> 2;
       }
+      //System.out.println("  next suffix=" + suffix + " versionAutoPrefix=" + versionAutoPrefix);
       startBytePos = suffixesReader.getPosition();
       ste.term.setLength(prefix + suffix);
       ste.term.grow(ste.term.length());

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/Document.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/Document.java?rev=1636528&r1=1636527&r2=1636528&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/Document.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/Document.java Tue Nov  4 09:31:24 2014
@@ -46,7 +46,6 @@ public final class Document implements I
 
   /** Constructs a new document with no fields. */
   public Document() {}
-  
 
   /**
    * <p>Adds a field to a document.  Several fields may be added with

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/Document2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/Document2.java?rev=1636528&r1=1636527&r2=1636528&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/Document2.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/Document2.java Tue Nov  4 09:31:24 2014
@@ -97,11 +97,8 @@ public class Document2 implements Iterab
 
     private TokenStream getReusedBinaryTokenStream(BytesRef value, TokenStream reuse) {
       BinaryTokenStream bts;
-      if (reuse != null) {
-        if (reuse instanceof BinaryTokenStream == false) {
-          // BUG
-          FieldTypes.illegalState(fieldName, "should have received BinaryTokenStream for reuse, but got " + reuse);
-        }
+      // It might be non-null and not a BinaryTokenStream if this is an atom field that just add a too small or too big term:
+      if (reuse != null && reuse instanceof BinaryTokenStream) {
         bts = (BinaryTokenStream) reuse;
       } else {
         bts = new BinaryTokenStream();
@@ -112,11 +109,8 @@ public class Document2 implements Iterab
 
     private TokenStream getReusedStringTokenStream(String value, TokenStream reuse) {
       StringTokenStream sts;
-      if (reuse != null) {
-        if (reuse instanceof StringTokenStream == false) {
-          // BUG
-          FieldTypes.illegalState(fieldName, "should have received StringTokenStream for reuse, but got " + reuse);
-        }
+      // It might be non-null and not a StringTokenStream if this is an atom field that just add a too small or too big term:
+      if (reuse != null && reuse instanceof StringTokenStream) {
         sts = (StringTokenStream) reuse;
       } else {
         sts = new StringTokenStream();
@@ -149,6 +143,21 @@ public class Document2 implements Iterab
       case DATE:
         return getReusedBinaryTokenStream(longToBytes(((Date) value).getTime()), reuse);
       case ATOM:
+        if (fieldType.minTokenLength != null) {
+          if (value instanceof String) {
+            String s = (String) value;
+            if (s.length() < fieldType.minTokenLength.intValue() ||
+                s.length() > fieldType.maxTokenLength.intValue()) {
+              return EMPTY_TOKEN_STREAM;
+            }
+          } else if (value instanceof BytesRef) {
+            BytesRef b = (BytesRef) value;
+            if (b.length < fieldType.minTokenLength.intValue() ||
+                b.length > fieldType.maxTokenLength.intValue()) {
+              return EMPTY_TOKEN_STREAM;
+            }
+          }
+        }
         if (value instanceof String) {
           return getReusedStringTokenStream((String) value, reuse);
         } else {
@@ -735,4 +744,11 @@ public class Document2 implements Iterab
     }
     return b.toString();
   }
+
+  private static final TokenStream EMPTY_TOKEN_STREAM = new TokenStream() {
+      @Override
+      public final boolean incrementToken() {
+        return false;
+      }
+    };
 }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/FieldTypes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/FieldTypes.java?rev=1636528&r1=1636527&r2=1636528&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/FieldTypes.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/FieldTypes.java Tue Nov  4 09:31:24 2014
@@ -99,7 +99,7 @@ import org.apache.lucene.util.Version;
 //   PerFieldAnalyzerWrapper
 //   oal.document
 
-// nocommit not null?
+// nocommit index field names the doc has?
 
 // nocommit fix simple qp to optionally take this?
 
@@ -120,10 +120,18 @@ import org.apache.lucene.util.Version;
 
 // nocommit getTermFilter?
 
+// nocommit facets?
+
+// nocommit live values?
+
+// nocommit expr fields?
+
 // nocommit default qp operator
 
 // nocommit copy field?
 
+// nocommit sort proxy field?
+
 // nocommit controlling compression of stored fields, norms
 
 // nocommit can we somehow detect at search time if the field types you are using doesn't match the searcher you are now searching against?
@@ -212,7 +220,7 @@ public class FieldTypes {
 
   // nocommit nested docs?
 
-  // nocommit required?
+  // nocommit required?  not null?
 
   /** Just like current oal.document.FieldType, except for each setting it can also record "not-yet-set". */
   static class FieldType implements IndexableFieldType {
@@ -374,6 +382,9 @@ public class FieldTypes {
         }
         break;
       case ATOM:
+        if (highlighted == Boolean.TRUE) {
+          illegalState(name, "type " + valueType + " cannot highlight");
+        }
         if (indexAnalyzer != null) {
           illegalState(name, "type " + valueType + " cannot have an indexAnalyzer");
         }
@@ -384,8 +395,7 @@ public class FieldTypes {
           illegalState(name, "type " + valueType + " cannot index norms");
         }
         if (indexOptions != IndexOptions.NONE && indexOptions.compareTo(IndexOptions.DOCS) > 0) {
-          // nocommit too anal?
-          illegalState(name, "type " + valueType + " can only be indexed as DOCS_ONLY; got " + indexOptions);
+          illegalState(name, "type " + valueType + " can only be indexed as DOCS; got " + indexOptions);
         }
         if (maxTokenCount != null) {
           illegalState(name, "type " + valueType + " cannot set max token count");
@@ -601,11 +611,11 @@ public class FieldTypes {
         b.append(fastRanges);
         if (fastRanges == Boolean.TRUE) {
           if (blockTreeMinItemsInAutoPrefix != null) {
-            b.append("  auto-prefix blocks: ");
+            b.append(" (auto-prefix blocks: ");
             b.append(blockTreeMinItemsInAutoPrefix);
             b.append(" - ");
             b.append(blockTreeMaxItemsInAutoPrefix);
-            b.append("\n");
+            b.append(")");
           }
         }
       } else {
@@ -3140,4 +3150,11 @@ public class FieldTypes {
       }
     }
   }
+
+  /** Defines a dynamic field, computed by a Javascript expression referring
+   *  to other field values, to be used for sorting. */
+  public void addIntExpressionField(String fieldName, String expression) {
+    // nocommit how to do this?  must we make a FieldTypes subclass in expressions module = pita?
+  }
+  // nocommit also long, float, double
 }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java?rev=1636528&r1=1636527&r2=1636528&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java Tue Nov  4 09:31:24 2014
@@ -17,14 +17,14 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.PriorityQueue;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.Bits;
-
 import java.io.IOException;
 import java.util.Arrays;
 
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.PriorityQueue;
+
 /**
  * Exposes {@link TermsEnum} API, merged from {@link TermsEnum} API of sub-segments.
  * This does a merge sort, by term text, of the sub-readers.
@@ -304,7 +304,6 @@ public final class MultiTermsEnum extend
     } else {
       current = null;
     }
-
     return current;
   }
 

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingTermVectorsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingTermVectorsFormat.java?rev=1636528&r1=1636527&r2=1636528&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingTermVectorsFormat.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingTermVectorsFormat.java Tue Nov  4 09:31:24 2014
@@ -1,12 +1,9 @@
 package org.apache.lucene.codecs.compressing;
 
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.FieldType;
-import org.apache.lucene.document.TextField;
-import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.document.Document2;
 import org.apache.lucene.index.BaseTermVectorsFormatTestCase;
+import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
@@ -45,10 +42,9 @@ public class TestCompressingTermVectorsF
   public void testNoOrds() throws Exception {
     Directory dir = newDirectory();
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
-    Document doc = new Document();
-    FieldType ft = new FieldType(TextField.TYPE_NOT_STORED);
-    ft.setStoreTermVectors(true);
-    doc.add(new Field("foo", "this is a test", ft));
+    Document2 doc = iw.newDocument();
+    iw.getFieldTypes().enableTermVectors("foo");
+    doc.addLargeText("foo", "this is a test");
     iw.addDocument(doc);
     LeafReader ir = getOnlySegmentReader(iw.getReader());
     Terms terms = ir.getTermVector(0, "foo");

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestAutoPrefixTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestAutoPrefixTerms.java?rev=1636528&r1=1636527&r2=1636528&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestAutoPrefixTerms.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestAutoPrefixTerms.java Tue Nov  4 09:31:24 2014
@@ -30,14 +30,13 @@ import org.apache.lucene.analysis.MockAn
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.document.BinaryDocValuesField;
-import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Document2;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
-import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
@@ -89,9 +88,9 @@ public class TestAutoPrefixTerms extends
     }
 
     for(String term : terms) {
-      Document doc = new Document();
-      doc.add(new StringField("field", term, Field.Store.NO));
-      doc.add(new NumericDocValuesField("field", Long.parseLong(term)));
+      Document2 doc = w.newDocument();
+      doc.addAtom("field", term);
+      doc.addLong("long", Long.parseLong(term));
       w.addDocument(doc);
     }
 
@@ -143,7 +142,7 @@ public class TestAutoPrefixTerms extends
                                                    maxTerm, maxInclusive);
 
       TermsEnum te = ca.getTermsEnum(MultiFields.getTerms(r, "field"));
-      NumericDocValues docValues = MultiDocValues.getNumericValues(r, "field");
+      NumericDocValues docValues = MultiDocValues.getNumericValues(r, "long");
       DocsEnum docsEnum = null;
 
       VerifyAutoPrefixTerms verifier = new VerifyAutoPrefixTerms(r.maxDoc(), minTerm, maxTerm);
@@ -201,9 +200,18 @@ public class TestAutoPrefixTerms extends
 
   // Numbers are encoded in full binary (4 byte ints):
   public void testBinaryNumericRanges() throws Exception {
+
+    if (VERBOSE) {
+      System.out.println("TEST: minItemsPerBlock=" + minItemsPerBlock);
+      System.out.println("TEST: maxItemsPerBlock=" + maxItemsPerBlock);
+      System.out.println("TEST: minTermsAutoPrefix=" + minTermsAutoPrefix);
+      System.out.println("TEST: maxTermsAutoPrefix=" + maxTermsAutoPrefix);
+    }
+
     Directory dir = newDirectory();
     IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
     iwc.setCodec(codec);
+    iwc.setMergeScheduler(new SerialMergeScheduler());
     IndexWriter w = new IndexWriter(dir, iwc);
     int numTerms = TestUtil.nextInt(random(), 3000, 50000);
     Set<Integer> terms = new HashSet<>();
@@ -212,13 +220,14 @@ public class TestAutoPrefixTerms extends
     }
 
     for(Integer term : terms) {
-      Document doc = new Document();
-      doc.add(new BinaryField("field", intToBytes(term)));
-      doc.add(new NumericDocValuesField("field", term));
+      Document2 doc = w.newDocument();
+      doc.addAtom("field", intToBytes(term));
+      doc.addInt("int", term.intValue());
       w.addDocument(doc);
     }
 
     if (random().nextBoolean()) {
+      if (VERBOSE) System.out.println("TEST: now force merge");
       w.forceMerge(1);
     }
 
@@ -265,7 +274,7 @@ public class TestAutoPrefixTerms extends
                                                    maxTerm, maxInclusive);
 
       TermsEnum te = ca.getTermsEnum(MultiFields.getTerms(r, "field"));
-      NumericDocValues docValues = MultiDocValues.getNumericValues(r, "field");
+      NumericDocValues docValues = MultiDocValues.getNumericValues(r, "int");
       DocsEnum docsEnum = null;
       VerifyAutoPrefixTerms verifier = new VerifyAutoPrefixTerms(r.maxDoc(), minTerm, maxTerm);
       while (te.next() != null) {
@@ -330,11 +339,12 @@ public class TestAutoPrefixTerms extends
     while (terms.size() < numTerms) {
       terms.add(TestUtil.randomSimpleString(random()));
     }
+    w.getFieldTypes().setDocValuesType("binary", DocValuesType.BINARY);
 
     for(String term : terms) {
-      Document doc = new Document();
-      doc.add(new StringField("field", term, Field.Store.NO));
-      doc.add(new BinaryDocValuesField("field", new BytesRef(term)));
+      Document2 doc = w.newDocument();
+      doc.addAtom("field", term);
+      doc.addBinary("binary", new BytesRef(term));
       w.addDocument(doc);
     }
 
@@ -379,7 +389,7 @@ public class TestAutoPrefixTerms extends
 
       CompiledAutomaton ca = new CompiledAutomaton(prefixBR);
       TermsEnum te = ca.getTermsEnum(MultiFields.getTerms(r, "field"));
-      BinaryDocValues docValues = MultiDocValues.getBinaryValues(r, "field");
+      BinaryDocValues docValues = MultiDocValues.getBinaryValues(r, "binary");
       DocsEnum docsEnum = null;
 
       VerifyAutoPrefixTerms verifier = new VerifyAutoPrefixTerms(r.maxDoc(), prefixBR);
@@ -436,11 +446,11 @@ public class TestAutoPrefixTerms extends
     int numDocs = 30;
 
     for(int i=0;i<numDocs;i++) {
-      Document doc = new Document();
-      doc.add(new StringField("field", "" + (char) (97+i), Field.Store.NO));
+      Document2 doc = w.newDocument();
+      doc.addAtom("field", "" + (char) (97+i));
       w.addDocument(doc);
-      doc = new Document();
-      doc.add(new StringField("field", "a" + (char) (97+i), Field.Store.NO));
+      doc = w.newDocument();
+      doc.addAtom("field", "a" + (char) (97+i));
       w.addDocument(doc);
     }
 

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestBlockPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestBlockPostingsFormat.java?rev=1636528&r1=1636527&r2=1636528&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestBlockPostingsFormat.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestBlockPostingsFormat.java Tue Nov  4 09:31:24 2014
@@ -21,8 +21,7 @@ import org.apache.lucene.analysis.MockAn
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.blocktree.FieldReader;
 import org.apache.lucene.codecs.blocktree.Stats;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Document2;
 import org.apache.lucene.index.BasePostingsFormatTestCase;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexWriter;
@@ -45,10 +44,11 @@ public class TestBlockPostingsFormat ext
   public void testFinalBlock() throws Exception {
     Directory d = newDirectory();
     IndexWriter w = new IndexWriter(d, new IndexWriterConfig(new MockAnalyzer(random())));
+    w.getFieldTypes().disableSorting("field");
     for(int i=0;i<25;i++) {
-      Document doc = new Document();
-      doc.add(newStringField("field", Character.toString((char) (97+i)), Field.Store.NO));
-      doc.add(newStringField("field", "z" + Character.toString((char) (97+i)), Field.Store.NO));
+      Document2 doc = w.newDocument();
+      doc.addAtom("field", Character.toString((char) (97+i)));
+      doc.addAtom("field", "z" + Character.toString((char) (97+i)));
       w.addDocument(doc);
     }
     w.forceMerge(1);

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestBlockPostingsFormat2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestBlockPostingsFormat2.java?rev=1636528&r1=1636527&r2=1636528&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestBlockPostingsFormat2.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestBlockPostingsFormat2.java Tue Nov  4 09:31:24 2014
@@ -18,15 +18,12 @@ package org.apache.lucene.codecs.lucene5
  */
 
 import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.FieldType;
-import org.apache.lucene.document.TextField;
+import org.apache.lucene.document.Document2;
+import org.apache.lucene.document.FieldTypes;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
@@ -64,72 +61,58 @@ public class TestBlockPostingsFormat2 ex
     super.tearDown();
   }
   
-  private Document newDocument() {
-    Document doc = new Document();
+  private Document2 newDocument(String contents) {
+    Document2 doc = iw.newDocument();
+    FieldTypes fieldTypes = iw.getFieldTypes();
     for (IndexOptions option : IndexOptions.values()) {
       if (option == IndexOptions.NONE) {
         continue;
       }
-      FieldType ft = new FieldType(TextField.TYPE_NOT_STORED);
+      String fieldName = option.toString();
       // turn on tvs for a cross-check, since we rely upon checkindex in this test (for now)
-      ft.setStoreTermVectors(true);
-      ft.setStoreTermVectorOffsets(true);
-      ft.setStoreTermVectorPositions(true);
-      ft.setStoreTermVectorPayloads(true);
-      ft.setIndexOptions(option);
-      doc.add(new Field(option.toString(), "", ft));
+      fieldTypes.disableHighlighting(fieldName);
+      fieldTypes.enableTermVectors(fieldName);
+      fieldTypes.enableTermVectorOffsets(fieldName);
+      fieldTypes.enableTermVectorPositions(fieldName);
+      fieldTypes.enableTermVectorPayloads(fieldName);
+      fieldTypes.setIndexOptions(fieldName, option);
+      doc.addLargeText(fieldName, contents.replaceAll("name", fieldName));
     }
     return doc;
   }
 
   /** tests terms with df = blocksize */
   public void testDFBlockSize() throws Exception {
-    Document doc = newDocument();
     for (int i = 0; i < Lucene50PostingsFormat.BLOCK_SIZE; i++) {
-      for (IndexableField f : doc.getFields()) {
-        ((Field) f).setStringValue(f.name() + " " + f.name() + "_2");
-      }
-      iw.addDocument(doc);
+      iw.addDocument(newDocument("name name_2"));
     }
   }
 
   /** tests terms with df % blocksize = 0 */
   public void testDFBlockSizeMultiple() throws Exception {
-    Document doc = newDocument();
     for (int i = 0; i < Lucene50PostingsFormat.BLOCK_SIZE * 16; i++) {
-      for (IndexableField f : doc.getFields()) {
-        ((Field) f).setStringValue(f.name() + " " + f.name() + "_2");
-      }
-      iw.addDocument(doc);
+      iw.addDocument(newDocument("name name_2"));
     }
   }
   
   /** tests terms with ttf = blocksize */
   public void testTTFBlockSize() throws Exception {
-    Document doc = newDocument();
     for (int i = 0; i < Lucene50PostingsFormat.BLOCK_SIZE/2; i++) {
-      for (IndexableField f : doc.getFields()) {
-        ((Field) f).setStringValue(f.name() + " " + f.name() + " " + f.name() + "_2 " + f.name() + "_2");
-      }
-      iw.addDocument(doc);
+      iw.addDocument(newDocument("name name name_2 name_2"));
     }
   }
   
   /** tests terms with ttf % blocksize = 0 */
   public void testTTFBlockSizeMultiple() throws Exception {
-    Document doc = newDocument();
+    String proto = "name name name name name_2 name_2 name_2 name_2";
+    StringBuilder val = new StringBuilder();
+    for (int j = 0; j < 16; j++) {
+      val.append(proto);
+      val.append(" ");
+    }
+    String pattern = val.toString();
     for (int i = 0; i < Lucene50PostingsFormat.BLOCK_SIZE/2; i++) {
-      for (IndexableField f : doc.getFields()) {
-        String proto = (f.name() + " " + f.name() + " " + f.name() + " " + f.name() + " " 
-                       + f.name() + "_2 " + f.name() + "_2 " + f.name() + "_2 " + f.name() + "_2");
-        StringBuilder val = new StringBuilder();
-        for (int j = 0; j < 16; j++) {
-          val.append(proto);
-          val.append(" ");
-        }
-        ((Field) f).setStringValue(val.toString());
-      }
-      iw.addDocument(doc);
+      iw.addDocument(newDocument(pattern));
     }
   }
 }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/document/TestDocument2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/document/TestDocument2.java?rev=1636528&r1=1636527&r2=1636528&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/document/TestDocument2.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/document/TestDocument2.java Tue Nov  4 09:31:24 2014
@@ -17,6 +17,7 @@ package org.apache.lucene.document;
  * limitations under the License.
  */
 
+import java.io.IOException;
 import java.io.StringReader;
 import java.net.InetAddress;
 import java.text.SimpleDateFormat;
@@ -46,6 +47,7 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.PhraseQuery;
+import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
@@ -1628,6 +1630,59 @@ public class TestDocument2 extends Lucen
     dir.close();
   }
 
+  public void testMinMaxAtom() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
+    FieldTypes fieldTypes = w.getFieldTypes();
+    fieldTypes.setMinMaxTokenLength("field", 2, 7);
+    fieldTypes.setMultiValued("field");
+    Document2 doc = w.newDocument();
+    doc.addAtom("field", "a");
+    doc.addAtom("field", "ab");
+    doc.addAtom("field", "goodbyeyou");
+    w.addDocument(doc);
+
+    DirectoryReader r = DirectoryReader.open(w, true);
+    fieldTypes = r.getFieldTypes();
+
+    IndexSearcher s = newSearcher(r);
+    assertEquals(0, hitCount(s, fieldTypes.newStringTermQuery("field", "a")));
+    assertEquals(1, hitCount(s, fieldTypes.newStringTermQuery("field", "ab")));
+    assertEquals(0, hitCount(s, fieldTypes.newStringTermQuery("field", "goodbyeyou")));
+    r.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testMinMaxBinaryAtom() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
+    FieldTypes fieldTypes = w.getFieldTypes();
+    fieldTypes.setMinMaxTokenLength("field", 2, 7);
+    fieldTypes.setMultiValued("field");
+    Document2 doc = w.newDocument();
+    doc.addAtom("field", new BytesRef(new byte[1]));
+    doc.addAtom("field", new BytesRef(new byte[2]));
+    doc.addAtom("field", new BytesRef(new byte[10]));
+    w.addDocument(doc);
+
+    DirectoryReader r = DirectoryReader.open(w, true);
+    fieldTypes = r.getFieldTypes();
+
+    IndexSearcher s = newSearcher(r);
+    assertEquals(0, hitCount(s, fieldTypes.newBinaryTermQuery("field", new byte[1])));
+    assertEquals(1, hitCount(s, fieldTypes.newBinaryTermQuery("field", new byte[2])));
+    assertEquals(0, hitCount(s, fieldTypes.newBinaryTermQuery("field", new byte[10])));
+    r.close();
+    w.close();
+    dir.close();
+  }
+
+  private static int hitCount(IndexSearcher s, Query q) throws IOException {
+    // TODO: use TotalHitCountCollector sometimes
+    return s.search(q, 1).totalHits;
+  }
+
   // nocommit test per-field analyzers
 
   // nocommit test per-field sims

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java?rev=1636528&r1=1636527&r2=1636528&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java Tue Nov  4 09:31:24 2014
@@ -21,9 +21,9 @@ import java.io.IOException;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Document2;
 import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.FieldTypes;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery;
@@ -41,7 +41,6 @@ import org.apache.lucene.store.MockDirec
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 
-
 public class TestOmitTf extends LuceneTestCase {
   
   public static class SimpleSimilarity extends TFIDFSimilarity {
@@ -74,29 +73,28 @@ public class TestOmitTf extends LuceneTe
     Directory ram = newDirectory();
     Analyzer analyzer = new MockAnalyzer(random());
     IndexWriter writer = new IndexWriter(ram, newIndexWriterConfig(analyzer));
-    Document d = new Document();
+    FieldTypes fieldTypes = writer.getFieldTypes();
+    Document2 d = writer.newDocument();
         
     // this field will have Tf
-    Field f1 = newField("f1", "This field has term freqs", normalType);
-    d.add(f1);
+    d.addLargeText("f1", "This field has term freqs");
        
     // this field will NOT have Tf
-    Field f2 = newField("f2", "This field has NO Tf in all docs", omitType);
-    d.add(f2);
+    fieldTypes.setIndexOptions("f2", IndexOptions.DOCS);
+    fieldTypes.disableHighlighting("f2");
+    d.addLargeText("f2", "This field has NO Tf in all docs");
         
     writer.addDocument(d);
     writer.forceMerge(1);
     // now we add another document which has term freq for field f2 and not for f1 and verify if the SegmentMerger
     // keep things constant
-    d = new Document();
+    d = writer.newDocument();
         
     // Reverse
-    f1 = newField("f1", "This field has term freqs", omitType);
-    d.add(f1);
-        
-    f2 = newField("f2", "This field has NO Tf in all docs", normalType);     
-    d.add(f2);
-        
+    fieldTypes.setIndexOptions("f1", IndexOptions.DOCS);
+    fieldTypes.disableHighlighting("f1");
+    d.addLargeText("f1", "This field has term freqs");
+    d.addLargeText("f2", "This field has NO Tf in all docs");
     writer.addDocument(d);
 
     // force merge
@@ -124,32 +122,36 @@ public class TestOmitTf extends LuceneTe
             setMaxBufferedDocs(3).
             setMergePolicy(newLogMergePolicy(2))
     );
-    Document d = new Document();
+
+    FieldTypes fieldTypes = writer.getFieldTypes();
+
+    Document2 d = writer.newDocument();
         
     // this field will have Tf
-    Field f1 = newField("f1", "This field has term freqs", normalType);
-    d.add(f1);
+    d.addLargeText("f1", "This field has term freqs");
        
     // this field will NOT have Tf
-    Field f2 = newField("f2", "This field has NO Tf in all docs", omitType);
-    d.add(f2);
+    fieldTypes.disableHighlighting("f2");
+    fieldTypes.setIndexOptions("f2", IndexOptions.DOCS);
+    d.addLargeText("f2", "This field has NO Tf in all docs");
 
-    for(int i=0;i<30;i++)
+    for(int i=0;i<30;i++) {
       writer.addDocument(d);
+    }
         
     // now we add another document which has term freq for field f2 and not for f1 and verify if the SegmentMerger
     // keep things constant
-    d = new Document();
-        
-    // Reverese
-    f1 = newField("f1", "This field has term freqs", omitType);
-    d.add(f1);
+    d = writer.newDocument();
         
-    f2 = newField("f2", "This field has NO Tf in all docs", normalType);     
-    d.add(f2);
+    // Reverse
+    fieldTypes.disableHighlighting("f1");
+    fieldTypes.setIndexOptions("f1", IndexOptions.DOCS);
+    d.addLargeText("f1", "This field has term freqs");
+    d.addLargeText("f2", "This field has NO Tf in all docs");
         
-    for(int i=0;i<30;i++)
+    for(int i=0;i<30;i++) {
       writer.addDocument(d);
+    }
         
     // force merge
     writer.forceMerge(1);
@@ -177,21 +179,25 @@ public class TestOmitTf extends LuceneTe
             setMaxBufferedDocs(10).
             setMergePolicy(newLogMergePolicy(2))
     );
-    Document d = new Document();
+    Document2 d = writer.newDocument();
+    FieldTypes fieldTypes = writer.getFieldTypes();
         
     // this field will have Tf
-    Field f1 = newField("f1", "This field has term freqs", normalType);
-    d.add(f1);
+    fieldTypes.disableHighlighting("f1");
+    d.addLargeText("f1", "This field has term freqs");
        
     // this field will NOT have Tf
-    Field f2 = newField("f2", "This field has NO Tf in all docs", omitType);
-    d.add(f2);
+    fieldTypes.disableHighlighting("f2");
+    fieldTypes.setIndexOptions("f2", IndexOptions.DOCS);
+    d.addLargeText("f2", "This field has NO Tf in all docs");
 
-    for(int i=0;i<5;i++)
+    for(int i=0;i<5;i++) {
       writer.addDocument(d);
+    }     
 
-    for(int i=0;i<20;i++)
+    for(int i=0;i<20;i++) {
       writer.addDocument(d);
+    }
 
     // force merge
     writer.forceMerge(1);
@@ -225,18 +231,20 @@ public class TestOmitTf extends LuceneTe
     }
     Analyzer analyzer = new MockAnalyzer(random());
     IndexWriter writer = new IndexWriter(ram, newIndexWriterConfig(analyzer)
-                                                .setMaxBufferedDocs(3)
-                                                .setMergePolicy(newLogMergePolicy()));
+                                         .setMaxBufferedDocs(3)
+                                         .setMergePolicy(newLogMergePolicy()));
     LogMergePolicy lmp = (LogMergePolicy) writer.getConfig().getMergePolicy();
     lmp.setMergeFactor(2);
     lmp.setNoCFSRatio(0.0);
-    Document d = new Document();
-        
-    Field f1 = newField("f1", "This field has term freqs", omitType);
-    d.add(f1);
+    Document2 d = writer.newDocument();
+    FieldTypes fieldTypes = writer.getFieldTypes();
+    fieldTypes.disableHighlighting("f1");
+    fieldTypes.setIndexOptions("f1", IndexOptions.DOCS);
+    d.addLargeText("f1", "This field has term freqs");
 
-    for(int i=0;i<30;i++)
+    for(int i=0;i<30;i++) {
       writer.addDocument(d);
+    }
 
     writer.commit();
 
@@ -244,12 +252,12 @@ public class TestOmitTf extends LuceneTe
     
     // now add some documents with positions, and check
     // there is no prox after full merge
-    d = new Document();
-    f1 = newTextField("f1", "This field has positions", Field.Store.NO);
-    d.add(f1);
+    d = writer.newDocument();
+    d.addLargeText("f1", "This field has positions");
     
-    for(int i=0;i<30;i++)
+    for(int i=0;i<30;i++) {
       writer.addDocument(d);
+    }
  
     // force merge
     writer.forceMerge(1);
@@ -273,16 +281,16 @@ public class TestOmitTf extends LuceneTe
     );
         
     StringBuilder sb = new StringBuilder(265);
+    FieldTypes fieldTypes = writer.getFieldTypes();
+    fieldTypes.disableHighlighting("noTf");
+    fieldTypes.setIndexOptions("noTf", IndexOptions.DOCS);
     String term = "term";
     for(int i = 0; i<30; i++){
-      Document d = new Document();
+      Document2 d = writer.newDocument();
       sb.append(term).append(" ");
       String content  = sb.toString();
-      Field noTf = newField("noTf", content + (i%2==0 ? "" : " notf"), omitType);
-      d.add(noTf);
-          
-      Field tf = newField("tf", content + (i%2==0 ? " tf" : ""), normalType);
-      d.add(tf);
+      d.addLargeText("noTf", content + (i%2==0 ? "" : " notf"));
+      d.addLargeText("tf", content + (i%2==0 ? " tf" : ""));
           
       writer.addDocument(d);
       //System.out.println(d);
@@ -448,12 +456,8 @@ public class TestOmitTf extends LuceneTe
     Directory dir = newDirectory();
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir,
         newIndexWriterConfig(new MockAnalyzer(random())));
-    Document doc = new Document();
-    FieldType ft = new FieldType(TextField.TYPE_NOT_STORED);
-    ft.setIndexOptions(IndexOptions.DOCS);
-    ft.freeze();
-    Field f = newField("foo", "bar", ft);
-    doc.add(f);
+    Document2 doc = iw.newDocument();
+    doc.addAtom("foo", "bar");
     iw.addDocument(doc);
     IndexReader ir = iw.getReader();
     iw.close();

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestPrefixQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestPrefixQuery.java?rev=1636528&r1=1636527&r2=1636528&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestPrefixQuery.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestPrefixQuery.java Tue Nov  4 09:31:24 2014
@@ -26,9 +26,7 @@ import java.util.Set;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.Document2;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.MultiFields;
@@ -54,8 +52,8 @@ public class TestPrefixQuery extends Luc
                                         "/Computers/Windows"};
     RandomIndexWriter writer = new RandomIndexWriter(random(), directory);
     for (int i = 0; i < categories.length; i++) {
-      Document doc = new Document();
-      doc.add(newStringField("category", categories[i], Field.Store.YES));
+      Document2 doc = writer.newDocument();
+      doc.addAtom("category", categories[i]);
       writer.addDocument(doc);
     }
     IndexReader reader = writer.getReader();
@@ -120,8 +118,8 @@ public class TestPrefixQuery extends Luc
     RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
 
     for (String term : randomTerms) {
-      Document doc = new Document();
-      doc.add(new StringField("field", term, Field.Store.NO));
+      Document2 doc = w.newDocument();
+      doc.addAtom("field", term);
       w.addDocument(doc);
     }
 

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestTermRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestTermRangeQuery.java?rev=1636528&r1=1636527&r2=1636528&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestTermRangeQuery.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestTermRangeQuery.java Tue Nov  4 09:31:24 2014
@@ -27,14 +27,12 @@ import java.util.Set;
 import org.apache.lucene.analysis.*;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.Document2;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Terms;
@@ -45,7 +43,6 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 
-
 public class TestTermRangeQuery extends LuceneTestCase {
 
   private int docCount = 0;
@@ -271,10 +268,10 @@ public class TestTermRangeQuery extends 
   }
 
   private void insertDoc(IndexWriter writer, String content) throws IOException {
-    Document doc = new Document();
+    Document2 doc = writer.newDocument();
 
-    doc.add(newStringField("id", "id" + docCount, Field.Store.YES));
-    doc.add(newTextField("content", content, Field.Store.NO));
+    doc.addAtom("id", "id" + docCount);
+    doc.addLargeText("content", content);
 
     writer.addDocument(doc);
     docCount++;
@@ -421,8 +418,8 @@ public class TestTermRangeQuery extends 
     RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
 
     for (String term : randomTerms) {
-      Document doc = new Document();
-      doc.add(new StringField("field", term, Field.Store.NO));
+      Document2 doc = w.newDocument();
+      doc.addAtom("field", term);
       w.addDocument(doc);
     }
 

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestWildcard.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestWildcard.java?rev=1636528&r1=1636527&r2=1636528&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestWildcard.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestWildcard.java Tue Nov  4 09:31:24 2014
@@ -17,19 +17,16 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import org.apache.lucene.document.Field;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.LuceneTestCase;
+import java.io.IOException;
+
 import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Document2;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.index.Terms;
-
-import java.io.IOException;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
 
 /**
  * TestWildcard tests the '*' and '?' wildcard characters.
@@ -244,8 +241,8 @@ public class TestWildcard
     Directory indexStore = newDirectory();
     RandomIndexWriter writer = new RandomIndexWriter(random(), indexStore);
     for (int i = 0; i < contents.length; ++i) {
-      Document doc = new Document();
-      doc.add(newTextField(field, contents[i], Field.Store.YES));
+      Document2 doc = writer.newDocument();
+      doc.addLargeText(field, contents[i]);
       writer.addDocument(doc);
     }
     writer.close();
@@ -343,8 +340,8 @@ public class TestWildcard
         newIndexWriterConfig(new MockAnalyzer(random()))
         .setMergePolicy(newLogMergePolicy()));
     for (int i = 0; i < docs.length; i++) {
-      Document doc = new Document();
-      doc.add(newTextField(field, docs[i], Field.Store.NO));
+      Document2 doc = iw.newDocument();
+      doc.addLargeText(field, docs[i]);
       iw.addDocument(doc);
     }
     iw.close();

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestBufferedIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestBufferedIndexInput.java?rev=1636528&r1=1636527&r2=1636528&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestBufferedIndexInput.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestBufferedIndexInput.java Tue Nov  4 09:31:24 2014
@@ -25,21 +25,19 @@ import java.util.List;
 import java.util.Random;
 
 import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Document2;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.Term;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.index.Term;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.TestUtil;
-import org.apache.lucene.util.ArrayUtil;
 
 public class TestBufferedIndexInput extends LuceneTestCase {
   
@@ -221,9 +219,9 @@ public class TestBufferedIndexInput exte
                 setMergePolicy(newLogMergePolicy(false))
         );
         for(int i=0;i<37;i++) {
-          Document doc = new Document();
-          doc.add(newTextField("content", "aaa bbb ccc ddd" + i, Field.Store.YES));
-          doc.add(newTextField("id", "" + i, Field.Store.YES));
+          Document2 doc = writer.newDocument();
+          doc.addLargeText("content", "aaa bbb ccc ddd" + i);
+          doc.addAtom("id", "" + i);
           writer.addDocument(doc);
         }
 

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestLockFactory.java?rev=1636528&r1=1636527&r2=1636528&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestLockFactory.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestLockFactory.java Tue Nov  4 09:31:24 2014
@@ -25,14 +25,13 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Document2;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.Term;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.index.Term;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TermQuery;
@@ -392,8 +391,8 @@ public class TestLockFactory extends Luc
     }
 
     private void addDoc(IndexWriter writer) throws IOException {
-        Document doc = new Document();
-        doc.add(newTextField("content", "aaa", Field.Store.NO));
+        Document2 doc = writer.newDocument();
+        doc.addLargeText("content", "aaa");
         writer.addDocument(doc);
     }
 }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java?rev=1636528&r1=1636527&r2=1636528&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java Tue Nov  4 09:31:24 2014
@@ -22,8 +22,7 @@ import java.nio.file.Path;
 import java.util.Random;
 
 import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Document2;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.util.BytesRef;
@@ -341,16 +340,12 @@ public class TestMultiMMap extends BaseD
       mmapDir.setUseUnmap(true);
     MockDirectoryWrapper dir = new MockDirectoryWrapper(random, mmapDir);
     RandomIndexWriter writer = new RandomIndexWriter(random, dir, newIndexWriterConfig(new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy()));
-    Document doc = new Document();
-    Field docid = newStringField("docid", "0", Field.Store.YES);
-    Field junk = newStringField("junk", "", Field.Store.YES);
-    doc.add(docid);
-    doc.add(junk);
     
     int numDocs = 100;
     for (int i = 0; i < numDocs; i++) {
-      docid.setStringValue("" + i);
-      junk.setStringValue(TestUtil.randomUnicodeString(random));
+      Document2 doc = writer.newDocument();
+      doc.addAtom("docid", "" + i);
+      doc.addAtom("junk", TestUtil.randomUnicodeString(random));
       writer.addDocument(doc);
     }
     IndexReader reader = writer.getReader();

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestRAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestRAMDirectory.java?rev=1636528&r1=1636527&r2=1636528&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestRAMDirectory.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestRAMDirectory.java Tue Nov  4 09:31:24 2014
@@ -23,13 +23,11 @@ import java.nio.file.Path;
 
 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.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.util.English;
 import org.apache.lucene.util.IOUtils;
@@ -55,10 +53,9 @@ public class TestRAMDirectory extends Ba
     IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(
         new MockAnalyzer(random())).setOpenMode(OpenMode.CREATE));
     // add some documents
-    Document doc = null;
     for (int i = 0; i < docsToAdd; i++) {
-      doc = new Document();
-      doc.add(newStringField("content", English.intToEnglish(i).trim(), Field.Store.YES));
+      Document2 doc = writer.newDocument();
+      doc.addAtom("content", English.intToEnglish(i).trim());
       writer.addDocument(doc);
     }
     assertEquals(docsToAdd, writer.maxDoc());
@@ -135,8 +132,8 @@ public class TestRAMDirectory extends Ba
         @Override
         public void run() {
           for (int j=1; j<docsPerThread; j++) {
-            Document doc = new Document();
-            doc.add(newStringField("sizeContent", English.intToEnglish(num*docsPerThread+j).trim(), Field.Store.YES));
+            Document2 doc = writer.newDocument();
+            doc.addAtom("sizeContent", English.intToEnglish(num*docsPerThread+j).trim());
             try {
               writer.addDocument(doc);
             } catch (IOException e) {

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestWindowsMMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestWindowsMMap.java?rev=1636528&r1=1636527&r2=1636528&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestWindowsMMap.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestWindowsMMap.java Tue Nov  4 09:31:24 2014
@@ -19,17 +19,15 @@ package org.apache.lucene.store;
 
 import java.nio.file.Path;
 
-import org.apache.lucene.document.Field;
-import org.apache.lucene.util.LuceneTestCase;
-
 import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Document2;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.util.LuceneTestCase;
 
 public class TestWindowsMMap extends LuceneTestCase {
   
@@ -80,8 +78,8 @@ public class TestWindowsMMap extends Luc
     int num = atLeast(1000);
     for(int dx = 0; dx < num; dx ++) {
       String f = randomField();
-      Document doc = new Document();
-      doc.add(newTextField("data", f, Field.Store.YES));  
+      Document2 doc = writer.newDocument();
+      doc.addLargeText("data", f);  
       writer.addDocument(doc);
     }
     

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java?rev=1636528&r1=1636527&r2=1636528&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java Tue Nov  4 09:31:24 2014
@@ -41,7 +41,7 @@ import java.util.concurrent.atomic.Atomi
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Document2;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
@@ -64,12 +64,12 @@ import org.apache.lucene.util.BytesRefBu
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.LineFileDocs;
+import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
-import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
-import org.apache.lucene.util.automaton.CompiledAutomaton;
 import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
 import org.apache.lucene.util.automaton.RegExp;
 import org.apache.lucene.util.fst.BytesRefFSTEnum.InputOutput;
 import org.apache.lucene.util.fst.FST.Arc;
@@ -853,10 +853,6 @@ public class TestFSTs extends LuceneTest
       }
       RandomIndexWriter w = new RandomIndexWriter(random(), dir,
                                                   newIndexWriterConfig(new MockAnalyzer(random())).setOpenMode(IndexWriterConfig.OpenMode.CREATE));
-      Document doc = new Document();
-      Field idField = newStringField("id", "", Field.Store.NO);
-      doc.add(idField);
-
       final int NUM_IDS = atLeast(200);
       //final int NUM_IDS = (int) (377 * (1.0+random.nextDouble()));
       if (VERBOSE) {
@@ -878,7 +874,8 @@ public class TestFSTs extends LuceneTest
           }
         }
         allIDs.add(idString);
-        idField.setStringValue(idString);
+        Document2 doc = w.newDocument();
+        doc.addAtom("id", idString);
         w.addDocument(doc);
       }
 
@@ -983,10 +980,6 @@ public class TestFSTs extends LuceneTest
 
     RandomIndexWriter w = new RandomIndexWriter(random(), dir,
                                                 newIndexWriterConfig(new MockAnalyzer(random())).setOpenMode(IndexWriterConfig.OpenMode.CREATE));
-    Document doc = new Document();
-    Field f = newStringField("field", "", Field.Store.NO);
-    doc.add(f);
-
     final int NUM_TERMS = (int) (1000*RANDOM_MULTIPLIER * (1+random().nextDouble()));
     if (VERBOSE) {
       System.out.println("TEST: NUM_TERMS=" + NUM_TERMS);
@@ -998,7 +991,8 @@ public class TestFSTs extends LuceneTest
     }
 
     for(String term : allTerms) {
-      f.setStringValue(term);
+      Document2 doc = w.newDocument();
+      doc.addAtom("field", term);
       w.addDocument(doc);
     }