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/03 10:22:24 UTC

svn commit: r1636293 - in /lucene/dev/branches/lucene6005/lucene/core/src: java/org/apache/lucene/document/ java/org/apache/lucene/index/ test/org/apache/lucene/document/

Author: mikemccand
Date: Mon Nov  3 09:22:24 2014
New Revision: 1636293

URL: http://svn.apache.org/r1636293
Log:
LUCENE-6005: add Date, InetAddress types; add min/maxTokenLength; add maxTokenCount; use ValueType.NONE not null; each FieldType now stores Luceneversion it was created by

Added:
    lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/FilteringTokenFilter.java
      - copied, changed from r1635912, lucene/dev/branches/lucene6005/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/FilteringTokenFilter.java
    lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/LengthFilter.java
      - copied, changed from r1635912, lucene/dev/branches/lucene6005/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LengthFilter.java
    lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/LimitTokenCountFilter.java   (with props)
Modified:
    lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/CoreKeywordTokenizer.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/Document2StoredFieldVisitor.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/ParallelCompositeReader.java
    lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java
    lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
    lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/document/TestDocument2.java

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/CoreKeywordTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/CoreKeywordTokenizer.java?rev=1636293&r1=1636292&r2=1636293&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/CoreKeywordTokenizer.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/CoreKeywordTokenizer.java Mon Nov  3 09:22:24 2014
@@ -23,7 +23,9 @@ import org.apache.lucene.analysis.Tokeni
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 
-// nocommit ... this is fork of KeywordTokenizer ... what to do ... must Document2 live outside Lucene core ...
+// nocommit ... this is privte fork of KeywordTokenizer ... should we move it to ../analysis and make it public?  same deal with
+// LengthFilter and LimitTokenCountFilter:
+
 final class CoreKeywordTokenizer extends Tokenizer {
   /** Default read buffer size */ 
   public static final int DEFAULT_BUFFER_SIZE = 256;
@@ -45,7 +47,7 @@ final class CoreKeywordTokenizer extends
   }
 
   @Override
-  public final boolean incrementToken() throws IOException {
+  public boolean incrementToken() throws IOException {
     if (!done) {
       clearAttributes();
       done = true;
@@ -67,7 +69,7 @@ final class CoreKeywordTokenizer extends
   }
   
   @Override
-  public final void end() throws IOException {
+  public void end() throws IOException {
     super.end();
     // set final offset 
     offsetAtt.setOffset(finalOffset, finalOffset);

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=1636293&r1=1636292&r2=1636293&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 Mon Nov  3 09:22:24 2014
@@ -19,7 +19,9 @@ package org.apache.lucene.document;
 
 import java.io.IOException;
 import java.io.Reader;
+import java.net.InetAddress;
 import java.util.ArrayList;
+import java.util.Date;
 import java.util.Iterator;
 import java.util.List;
 
@@ -27,10 +29,10 @@ import org.apache.lucene.analysis.Analyz
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.document.FieldTypes.FieldType;
 import org.apache.lucene.index.DocValuesType;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.IndexableFieldType;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.FilterIterator;
 
 // nocommit clearly spell out which field defaults to what settings, e.g. that atom is not sorted by default
 
@@ -131,7 +133,10 @@ public class Document2 implements Iterab
         throw new IllegalArgumentException("analyzer must be the instance from FieldTypes: got " + analyzerIn + " vs " + analyzer);
       }
 
+      assert fieldTypes.getIndexOptions(fieldName) != IndexOptions.NONE;
+
       FieldTypes.FieldType fieldType = fieldTypes.getFieldType(fieldName);
+
       switch (fieldType.valueType) {
       case INT:
         return getReusedBinaryTokenStream(intToBytes(((Number) value).intValue()), reuse);
@@ -141,6 +146,8 @@ public class Document2 implements Iterab
         return getReusedBinaryTokenStream(longToBytes(((Number) value).longValue()), reuse);
       case DOUBLE:
         return getReusedBinaryTokenStream(longToBytes(Double.doubleToLongBits(((Number) value).doubleValue())), reuse);
+      case DATE:
+        return getReusedBinaryTokenStream(longToBytes(((Date) value).getTime()), reuse);
       case ATOM:
         if (value instanceof String) {
           return getReusedStringTokenStream((String) value, reuse);
@@ -150,18 +157,36 @@ public class Document2 implements Iterab
         }
 
       case BINARY:
-        assert value instanceof BytesRef;
-        BinaryTokenStream bts;
-        if (reuse != null) {
-          if (reuse instanceof BinaryTokenStream == false) {
-            FieldTypes.illegalState(fieldName, "should have had BinaryTokenStream for reuse, but got " + reuse);
+        {
+          assert value instanceof BytesRef;
+          BinaryTokenStream bts;
+          if (reuse != null) {
+            if (reuse instanceof BinaryTokenStream == false) {
+              FieldTypes.illegalState(fieldName, "should have had BinaryTokenStream for reuse, but got " + reuse);
+            }
+            bts = (BinaryTokenStream) reuse;
+          } else {
+            bts = new BinaryTokenStream();
           }
-          bts = (BinaryTokenStream) reuse;
-        } else {
-          bts = new BinaryTokenStream();
+          bts.setValue((BytesRef) value);
+          return bts;
+        }
+
+      case INET_ADDRESS:
+        {
+          assert value instanceof InetAddress;
+          BinaryTokenStream bts;
+          if (reuse != null) {
+            if (reuse instanceof BinaryTokenStream == false) {
+              FieldTypes.illegalState(fieldName, "should have had BinaryTokenStream for reuse, but got " + reuse);
+            }
+            bts = (BinaryTokenStream) reuse;
+          } else {
+            bts = new BinaryTokenStream();
+          }
+          bts.setValue(new BytesRef(((InetAddress) value).getAddress()));
+          return bts;
         }
-        bts.setValue((BytesRef) value);
-        return bts;
 
       case SHORT_TEXT:
       case TEXT:
@@ -203,6 +228,8 @@ public class Document2 implements Iterab
       case FLOAT:
       case DOUBLE:
         return (Number) value;
+      case DATE:
+        return ((Date) value).getTime();
       case BOOLEAN:
         if (value == Boolean.TRUE) {
           return Integer.valueOf(1);
@@ -228,6 +255,8 @@ public class Document2 implements Iterab
         return Integer.valueOf(Float.floatToIntBits((Float) value));
       case DOUBLE:
         return Long.valueOf(Double.doubleToLongBits((Double) value));
+      case DATE:
+        return Long.valueOf(((Date) value).getTime());
       case BOOLEAN:
         if (value == Boolean.TRUE) {
           return Integer.valueOf(1);
@@ -284,6 +313,8 @@ public class Document2 implements Iterab
           bytes[0] = 1;
         }
         return new BytesRef(bytes);
+      } else if (fieldType.valueType == FieldTypes.ValueType.INET_ADDRESS) {
+        return new BytesRef(((InetAddress) value).getAddress());
       } else if (value instanceof BytesRef) {
         return (BytesRef) value;
       } else {
@@ -295,9 +326,13 @@ public class Document2 implements Iterab
     public BytesRef binaryDocValue() {
       if (value instanceof BytesRef) {
         return (BytesRef) value;
-      } else if (value instanceof String && (fieldType.docValuesType == DocValuesType.BINARY || fieldType.docValuesType == DocValuesType.SORTED || fieldType.docValuesType == DocValuesType.SORTED_SET)) {
-        // nocommit somewhat evil we utf8-encode your string?
-        return new BytesRef((String) value);
+      } else if (fieldType.docValuesType == DocValuesType.BINARY || fieldType.docValuesType == DocValuesType.SORTED || fieldType.docValuesType == DocValuesType.SORTED_SET) {
+        if (fieldType.valueType == FieldTypes.ValueType.INET_ADDRESS) {
+          return new BytesRef(((InetAddress) value).getAddress());
+        } else if (value instanceof String) {
+          // nocommit somewhat evil we utf8-encode your string?
+          return new BytesRef((String) value);
+        }
       }
 
       return null;
@@ -333,7 +368,6 @@ public class Document2 implements Iterab
   }
 
   public IndexableField getField(String name) {
-    List<IndexableField> result = new ArrayList<>();
     for (FieldValue field : fields) {
       if (field.name().equals(name)) {
         return field;
@@ -484,9 +518,25 @@ public class Document2 implements Iterab
     fields.add(new FieldValue(fieldName, Boolean.valueOf(value)));
   }
 
+  public void addDate(String fieldName, Date value) {
+    if (changeSchema) {
+      fieldTypes.recordValueType(fieldName, FieldTypes.ValueType.DATE);
+    }
+    fields.add(new FieldValue(fieldName, value));
+  }
+
+  /** Add an {@code InetAddress} field.  This is indexed as a binary atom under the hood, for sorting,
+   *  range filtering and stored. */
+  public void addInetAddress(String fieldName, InetAddress value) {
+    if (changeSchema) {
+      fieldTypes.recordValueType(fieldName, FieldTypes.ValueType.INET_ADDRESS);
+    }
+    fields.add(new FieldValue(fieldName, value));
+  }
+
   static {
     // nocommit is there a cleaner/general way to detect missing enum value in case switch statically?  must we use ecj?
-    assert FieldTypes.ValueType.values().length == 9: "missing case for switch statement below";
+    assert FieldTypes.ValueType.values().length == 12: "missing case for switch statement below";
   }
 
   /** Note: this FieldTypes must already know about all the fields in the incoming doc. */
@@ -529,6 +579,12 @@ public class Document2 implements Iterab
       case BOOLEAN:
         addBoolean(fieldName, ((Boolean) field.value).booleanValue());
         break;
+      case DATE:
+        addDate(fieldName, (Date) field.value);
+        break;
+      case INET_ADDRESS:
+        addInetAddress(fieldName, (InetAddress) field.value);
+        break;
       default:
         // BUG:
         throw new AssertionError("missing valueType=" + fieldType.valueType + " in switch");
@@ -585,6 +641,28 @@ public class Document2 implements Iterab
     return null;
   }
 
+  public Date getDate(String fieldName) {
+    // nocommit can we assert this is a known field and that its type is date...?
+    for(FieldValue fieldValue : fields) {
+      if (fieldValue.fieldName.equals(fieldName)) {
+        return (Date) fieldValue.value;
+      }
+    }
+
+    return null;
+  }
+
+  public InetAddress getInetAddress(String fieldName) {
+    // nocommit can we assert this is a known field and that its type is inet address...?
+    for(FieldValue fieldValue : fields) {
+      if (fieldValue.fieldName.equals(fieldName)) {
+        return (InetAddress) fieldValue.value;
+      }
+    }
+
+    return null;
+  }
+
   public String getString(String fieldName) {
     // nocommit can we assert this is a known field and that its type is text/short_text...?
     for(FieldValue fieldValue : fields) {

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/Document2StoredFieldVisitor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/Document2StoredFieldVisitor.java?rev=1636293&r1=1636292&r2=1636293&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/Document2StoredFieldVisitor.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/Document2StoredFieldVisitor.java Mon Nov  3 09:22:24 2014
@@ -18,6 +18,8 @@ package org.apache.lucene.document;
  */
 
 import java.io.IOException;
+import java.net.InetAddress;
+import java.util.Date;
 import java.util.HashSet;
 import java.util.Set;
 
@@ -79,7 +81,12 @@ public class Document2StoredFieldVisitor
 
   @Override
   public void binaryField(FieldInfo fieldInfo, byte[] value) throws IOException {
-    doc.addBinary(fieldInfo.name, new BytesRef(value));
+    FieldTypes.FieldType fieldType = getFieldType(fieldInfo.name);
+    if (fieldType != null && fieldType.valueType == FieldTypes.ValueType.INET_ADDRESS) {
+      doc.addInetAddress(fieldInfo.name, InetAddress.getByAddress(value));
+    } else {
+      doc.addBinary(fieldInfo.name, new BytesRef(value));
+    }
   }
 
   @Override
@@ -101,7 +108,12 @@ public class Document2StoredFieldVisitor
 
   @Override
   public void longField(FieldInfo fieldInfo, long value) {
-    doc.addLong(fieldInfo.name, value);
+    FieldTypes.FieldType fieldType = getFieldType(fieldInfo.name);
+    if (fieldType != null && fieldType.valueType == FieldTypes.ValueType.DATE) {
+      doc.addDate(fieldInfo.name, new Date(value));
+    } else {
+      doc.addLong(fieldInfo.name, value);
+    }
   }
 
   @Override

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=1636293&r1=1636292&r2=1636293&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 Mon Nov  3 09:22:24 2014
@@ -18,14 +18,18 @@ package org.apache.lucene.document;
  */
 
 import java.io.IOException;
+import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.AnalyzerWrapper;
 import org.apache.lucene.analysis.DelegatingAnalyzerWrapper;
+import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.DocValuesFormat;
@@ -43,7 +47,6 @@ import org.apache.lucene.index.Indexable
 import org.apache.lucene.index.SegmentInfos;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.NumericRangeQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
@@ -62,16 +65,12 @@ import org.apache.lucene.store.RAMFile;
 import org.apache.lucene.store.RAMInputStream;
 import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.util.Version;
 
 // TODO
 //   - explore what it'd be like to add other higher level types?
 //     - BigInt, BigDecimal, IPV6
-//   - setters for posinc/offset gaps?
-//     - can we remove analyzer from IW?
 //   - what about sparse fields... anything for us to do...
 //   - payloads just stay write once in their own way?
 //   - how to handle old indices w/ no field types yet?
@@ -84,17 +83,10 @@ import org.apache.lucene.util.Version;
 //     - query parsers
 //       - exc if invalid field name asked for
 //       - numeric range queries "just work"
-//     - creating SortField
 //     - creating queries, catching invalid field names, no positions indexed, etc.
-//     - SortField should verify FieldTypes.sortable is set for that field
 //     - prox queries can verify field was indexed w/ positions
-//     - normal queries can verify field was even indexed
 //   - move analyzer out of IW/IWC into Field/FieldType/s only?
-//   - we could go back to allowing pulling the Document from a reader, updating, re-indexing?  e.g. we can know all fields were stored, and
-//     throw exc if not
-//   - get save/load working
 //   - why does STS fill offset...
-//   - no more field reuse right?
 
 // Lucene's secret schemas
 //   StoredFieldsVisitor
@@ -107,11 +99,11 @@ import org.apache.lucene.util.Version;
 //   PerFieldAnalyzerWrapper
 //   oal.document
 
-// nocommit optimize term range query when it's really "matches all docs"
+// nocommit not null?
 
-// nocommit add test: make sure you can suddenly turn on DV if they were off at first
+// nocommit fix simple qp to optionally take this?
 
-// nocommit each field type should record Version when it was added
+// nocommit optimize term range query when it's really "matches all docs"?
 
 // nocommit but how can we randomized IWC for tests?
 
@@ -120,12 +112,8 @@ import org.apache.lucene.util.Version;
 
 // nocommit move to oal.index?
 
-// nocommit maxTokenLength?
-
 // nocommit per-field norms format?  then we can commit these "tradeoffs"
 
-// nocommit sortMissingFirst/Last
-
 // nocommit default value?
 
 // nocommit can we have test infra that randomly reopens writer?
@@ -142,8 +130,6 @@ import org.apache.lucene.util.Version;
 
 // nocommit back compat: how to handle pre-schema indices
 
-// nocommit should we try to have "garbage collection" here?
-
 // nocommit maybe have a settable global default for "stored"?
 
 // nocommit can/should we validate field names here?
@@ -162,26 +148,20 @@ import org.apache.lucene.util.Version;
 
 // nocommit index-time sorting should be here too
 
-// nocommit sort by languages
-
 // nocommit can we require use of analyzer factories?
 
-// nocommit what schema options does solr offer
+// nocommit what schema options does solr/ES offer
 
 // nocommit accent removal and lowercasing for wildcards should just work
 
 // separate analyzer for phrase queries in suggesters
 
-// go through the process of adding a "new high schema type"
-
 // nocommit Index class?  enforcing unique id, xlog?
 
 // nocommit how to randomize IWC?  RIW?
 
 // nocommit unique/primary key ?
 
-// nocommit must document / make sugar for creating IndexSearcher w/ sim from this class
-
 // nocommit fix all change methods to call validate / rollback
 
 // nocommit float16?
@@ -198,6 +178,7 @@ public class FieldTypes {
   public static final String FIELD_PROPERTIES_KEY = "field_properties";
 
   enum ValueType {
+    NONE,
     TEXT,
     SHORT_TEXT,
     ATOM,  // nocommit binary sort of overlaps w/ this?
@@ -207,8 +188,9 @@ public class FieldTypes {
     DOUBLE,
     BINARY, // nocommit rename to bytes?
     BOOLEAN,
+    DATE,
+    INET_ADDRESS,
     // nocommit primary_key?
-    // nocommit inet addr?
   }
 
   private final boolean readOnly;
@@ -236,11 +218,20 @@ public class FieldTypes {
   static class FieldType implements IndexableFieldType {
     private final String name;
 
+    // Lucene version when we were created:
+    private final Version createdVersion;
+
     public FieldType(String name) {
+      this(name, Version.LATEST);
+    }
+
+    public FieldType(String name, Version version) {
       this.name = name;
+      this.createdVersion = version;
     }
 
-    volatile ValueType valueType;
+    // nocommit don't use null here:
+    volatile ValueType valueType = ValueType.NONE;
     volatile DocValuesType docValuesType = DocValuesType.NONE;
     private volatile boolean docValuesTypeSet;
 
@@ -254,13 +245,21 @@ public class FieldTypes {
     volatile Integer analyzerPositionGap;
     volatile Integer analyzerOffsetGap;
 
+    // Min/max token length, or null if there are no limits:
+    volatile Integer minTokenLength;
+    volatile Integer maxTokenLength;
+
+    // Limit on number of tokens to index for this field
+    volatile Integer maxTokenCount;
+    volatile Boolean consumeAllTokens;
+
     // Whether this field's values are stored, or null if it's not yet set:
     private volatile Boolean stored;
 
     // Whether this field's values should be indexed for sorting (using doc values):
     private volatile Boolean sortable;
     private volatile Boolean sortReversed;
-    private volatile Boolean sortMissingLast;
+    private volatile Boolean sortMissingLast = Boolean.TRUE;
 
     // Whether this field's values should be indexed for fast ranges (using numeric field for now):
     private volatile Boolean fastRanges;
@@ -293,97 +292,128 @@ public class FieldTypes {
     private volatile Analyzer indexAnalyzer;
     private volatile Similarity similarity;
 
+    private volatile Analyzer wrappedIndexAnalyzer;
+    private volatile Analyzer wrappedQueryAnalyzer;
+
     boolean validate() {
-      if (valueType != null) {
-        switch (valueType) {
-        case INT:
-        case FLOAT:
-        case LONG:
-        case DOUBLE:
-          if (highlighted == Boolean.TRUE) {
-            illegalState(name, "type " + valueType + " cannot highlight");
-          }
-          if (indexAnalyzer != null) {
-            illegalState(name, "type " + valueType + " cannot have an indexAnalyzer");
-          }
-          if (queryAnalyzer != null) {
-            illegalState(name, "type " + valueType + " cannot have a queryAnalyzer");
-          }
-          if (docValuesType != DocValuesType.NONE && (docValuesType != DocValuesType.NUMERIC && docValuesType != DocValuesType.SORTED_NUMERIC)) {
-            illegalState(name, "type " + valueType + " must use NUMERIC docValuesType (got: " + docValuesType + ")");
-          }
-          if (indexOptions != IndexOptions.NONE && indexOptions.compareTo(IndexOptions.DOCS) > 0) {
-            illegalState(name, "type " + valueType + " cannot use indexOptions > DOCS (got indexOptions " + indexOptions + ")");
-          }
-          if (indexNorms == Boolean.TRUE) {
-            illegalState(name, "type " + valueType + " cannot index norms");
-          }
-          break;
-        case TEXT:
-          if (sortable == Boolean.TRUE) {
-            illegalState(name, "type " + valueType + " cannot sort");
-          }
-          if (fastRanges == Boolean.TRUE) {
-            illegalState(name, "type " + valueType + " cannot optimize for range queries");
-          }
-          if (docValuesType != DocValuesType.NONE) {
-            illegalState(name, "type " + valueType + " cannot use docValuesType " + docValuesType);
-          }
-          break;
-        case SHORT_TEXT:
-          if (docValuesType != DocValuesType.NONE && docValuesType != DocValuesType.BINARY && docValuesType != DocValuesType.SORTED && docValuesType != DocValuesType.SORTED_SET) {
-            illegalState(name, "type " + valueType + " cannot use docValuesType " + docValuesType);
-          }
-          if (fastRanges == Boolean.TRUE) {
-            illegalState(name, "type " + valueType + " cannot optimize for range queries");
-          }
-          break;
-        case BINARY:
-          if (highlighted == Boolean.TRUE) {
-            illegalState(name, "type " + valueType + " cannot highlight");
-          }
-          if (indexAnalyzer != null) {
-            illegalState(name, "type " + valueType + " cannot have an indexAnalyzer");
-          }
-          if (queryAnalyzer != null) {
-            illegalState(name, "type " + valueType + " cannot have a queryAnalyzer");
-          }
-          if (docValuesType != DocValuesType.NONE && docValuesType != DocValuesType.BINARY && docValuesType != DocValuesType.SORTED && docValuesType != DocValuesType.SORTED_SET) {
-            illegalState(name, "type " + valueType + " must use BINARY, SORTED or SORTED_SET docValuesType (got: " + docValuesType + ")");
-          }
-          break;
-        case ATOM:
-          if (indexAnalyzer != null) {
-            illegalState(name, "type " + valueType + " cannot have an indexAnalyzer");
-          }
-          if (queryAnalyzer != null) {
-            illegalState(name, "type " + valueType + " cannot have a queryAnalyzer");
-          }
-          if (indexNorms == Boolean.TRUE) {
-            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);
-          }
-          break;
-        case BOOLEAN:
-          if (highlighted == Boolean.TRUE) {
-            illegalState(name, "type " + valueType + " cannot highlight");
-          }
-          if (indexNorms == Boolean.TRUE) {
-            illegalState(name, "type " + valueType + " cannot index norms");
-          }
-          if (docValuesType != DocValuesType.NONE && docValuesType != DocValuesType.NUMERIC && docValuesType != DocValuesType.SORTED_NUMERIC) {
-            illegalState(name, "type " + valueType + " must use NUMERIC or SORTED_NUMERIC docValuesType (got: " + docValuesType + ")");
-          }
-          break;
-        default:
-          throw new AssertionError("missing value type in switch");
+      switch (valueType) {
+      case NONE:
+        break;
+      case INT:
+      case FLOAT:
+      case LONG:
+      case DOUBLE:
+      case DATE:
+        if (highlighted == Boolean.TRUE) {
+          illegalState(name, "type " + valueType + " cannot highlight");
+        }
+        if (indexAnalyzer != null) {
+          illegalState(name, "type " + valueType + " cannot have an indexAnalyzer");
+        }
+        if (queryAnalyzer != null) {
+          illegalState(name, "type " + valueType + " cannot have a queryAnalyzer");
+        }
+        if (docValuesType != DocValuesType.NONE && (docValuesType != DocValuesType.NUMERIC && docValuesType != DocValuesType.SORTED_NUMERIC)) {
+          illegalState(name, "type " + valueType + " must use NUMERIC docValuesType (got: " + docValuesType + ")");
+        }
+        if (indexOptions != IndexOptions.NONE && indexOptions.compareTo(IndexOptions.DOCS) > 0) {
+          illegalState(name, "type " + valueType + " cannot use indexOptions > DOCS (got indexOptions " + indexOptions + ")");
+        }
+        if (indexNorms == Boolean.TRUE) {
+          illegalState(name, "type " + valueType + " cannot index norms");
+        }
+        if (minTokenLength != null) {
+          illegalState(name, "type " + valueType + " cannot set min/max token length");
+        }
+        if (maxTokenCount != null) {
+          illegalState(name, "type " + valueType + " cannot set max token count");
+        }
+        break;
+      case TEXT:
+        if (sortable == Boolean.TRUE) {
+          illegalState(name, "type " + valueType + " cannot sort");
+        }
+        if (fastRanges == Boolean.TRUE) {
+          illegalState(name, "type " + valueType + " cannot optimize for range queries");
+        }
+        if (docValuesType != DocValuesType.NONE) {
+          illegalState(name, "type " + valueType + " cannot use docValuesType " + docValuesType);
+        }
+        break;
+      case SHORT_TEXT:
+        if (docValuesType != DocValuesType.NONE && docValuesType != DocValuesType.BINARY && docValuesType != DocValuesType.SORTED && docValuesType != DocValuesType.SORTED_SET) {
+          illegalState(name, "type " + valueType + " cannot use docValuesType " + docValuesType);
+        }
+        if (fastRanges == Boolean.TRUE) {
+          illegalState(name, "type " + valueType + " cannot optimize for range queries");
+        }
+        break;
+      case BINARY:
+      case INET_ADDRESS:
+        if (highlighted == Boolean.TRUE) {
+          illegalState(name, "type " + valueType + " cannot highlight");
+        }
+        if (indexAnalyzer != null) {
+          illegalState(name, "type " + valueType + " cannot have an indexAnalyzer");
+        }
+        if (queryAnalyzer != null) {
+          illegalState(name, "type " + valueType + " cannot have a queryAnalyzer");
+        }
+        if (docValuesType != DocValuesType.NONE && docValuesType != DocValuesType.BINARY && docValuesType != DocValuesType.SORTED && docValuesType != DocValuesType.SORTED_SET) {
+          illegalState(name, "type " + valueType + " must use BINARY, SORTED or SORTED_SET docValuesType (got: " + docValuesType + ")");
+        }
+        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);
         }
-        // nocommit more checks
+        if (minTokenLength != null) {
+          illegalState(name, "type " + valueType + " cannot set min/max token length");
+        }
+        if (maxTokenCount != null) {
+          illegalState(name, "type " + valueType + " cannot set max token count");
+        }
+        break;
+      case ATOM:
+        if (indexAnalyzer != null) {
+          illegalState(name, "type " + valueType + " cannot have an indexAnalyzer");
+        }
+        if (queryAnalyzer != null) {
+          illegalState(name, "type " + valueType + " cannot have a queryAnalyzer");
+        }
+        if (indexNorms == Boolean.TRUE) {
+          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);
+        }
+        if (maxTokenCount != null) {
+          illegalState(name, "type " + valueType + " cannot set max token count");
+        }
+        break;
+      case BOOLEAN:
+        if (highlighted == Boolean.TRUE) {
+          illegalState(name, "type " + valueType + " cannot highlight");
+        }
+        if (indexNorms == Boolean.TRUE) {
+          illegalState(name, "type " + valueType + " cannot index norms");
+        }
+        if (docValuesType != DocValuesType.NONE && docValuesType != DocValuesType.NUMERIC && docValuesType != DocValuesType.SORTED_NUMERIC) {
+          illegalState(name, "type " + valueType + " must use NUMERIC or SORTED_NUMERIC docValuesType (got: " + docValuesType + ")");
+        }
+        if (minTokenLength != null) {
+          illegalState(name, "type " + valueType + " cannot set min/max token length");
+        }
+        if (maxTokenCount != null) {
+          illegalState(name, "type " + valueType + " cannot set max token count");
+        }
+        break;
+      default:
+        throw new AssertionError("missing value type in switch");
       }
 
+      // nocommit more checks
+
       if (multiValued == Boolean.TRUE &&
           (docValuesType == DocValuesType.NUMERIC ||
            docValuesType == DocValuesType.SORTED ||
@@ -459,6 +489,57 @@ public class FieldTypes {
       return true;
     }
 
+    private boolean needsWrapping() {
+      return minTokenLength != null || maxTokenCount != null;
+    }
+
+    void reWrapAnalyzers(Analyzer defaultIndexAnalyzer, Analyzer defaultQueryAnalyzer) {
+      // nocommit need test to verify wrapping for ATOM fields works correctly
+      if (needsWrapping()) {
+        if (indexAnalyzer != null) {
+          wrappedIndexAnalyzer = wrapAnalyzer(indexAnalyzer);
+        } else if (defaultIndexAnalyzer != null) {
+          wrappedIndexAnalyzer = wrapAnalyzer(defaultIndexAnalyzer);
+        } else {
+          wrappedIndexAnalyzer = null;
+        }
+        if (queryAnalyzer != null) {
+          wrappedQueryAnalyzer = wrapAnalyzer(queryAnalyzer);
+        } else if (defaultQueryAnalyzer != null) {
+          wrappedQueryAnalyzer = wrapAnalyzer(defaultQueryAnalyzer);
+        } else {
+          wrappedQueryAnalyzer = null;
+        }
+      } else {
+        wrappedIndexAnalyzer = null;
+        wrappedQueryAnalyzer = null;
+      }
+    }
+
+    private Analyzer wrapAnalyzer(final Analyzer in) {
+      return new AnalyzerWrapper(in.getReuseStrategy()) {
+        @Override
+        protected Analyzer getWrappedAnalyzer(String fieldName) {
+          return in;
+        }
+
+        @Override
+        protected TokenStreamComponents wrapComponents(String fieldName, TokenStreamComponents components) {
+          TokenStream end = components.getTokenStream();
+          if (minTokenLength != null) {
+            end = new LengthFilter(end,
+                                   minTokenLength.intValue(),
+                                   maxTokenLength.intValue());
+          }
+          if (maxTokenCount != null) {
+            end = new LimitTokenCountFilter(end, maxTokenCount.intValue(), consumeAllTokens.booleanValue());
+          }
+
+          return new TokenStreamComponents(components.getTokenizer(), end);
+        }
+      };
+    }
+
     @Override
     public String toString() {
       StringBuilder b = new StringBuilder();
@@ -466,11 +547,7 @@ public class FieldTypes {
       b.append(name);
       b.append("\":\n");
       b.append("  valueType: ");
-      if (valueType != null) {
-        b.append(valueType);
-      } else {
-        b.append("unset");
-      }
+      b.append(valueType);
       b.append('\n');
 
       if (blockTreeMinItemsInBlock != null) {
@@ -480,13 +557,6 @@ public class FieldTypes {
         b.append(blockTreeMaxItemsInBlock);
       }
 
-      if (blockTreeMinItemsInAutoPrefix != null) {
-        b.append("  auto-prefix blocks: ");
-        b.append(blockTreeMinItemsInAutoPrefix);
-        b.append(" - ");
-        b.append(blockTreeMaxItemsInAutoPrefix);
-      }
-
       if (analyzerPositionGap != null) {
         b.append("  multi-valued position gap: ");
         b.append(analyzerPositionGap);
@@ -517,9 +587,9 @@ public class FieldTypes {
           b.append(sortReversed);
         }
         if (sortMissingLast == Boolean.TRUE) {
-          b.append(" missing=last");
+          b.append(" (missing: last)");
         } else if (sortMissingLast == Boolean.FALSE) {
-          b.append(" missing=first");
+          b.append(" (missing: first)");
         }
       } else {
         b.append("unset");
@@ -529,6 +599,15 @@ public class FieldTypes {
       b.append("  fastRanges: ");
       if (fastRanges != null) {
         b.append(fastRanges);
+        if (fastRanges == Boolean.TRUE) {
+          if (blockTreeMinItemsInAutoPrefix != null) {
+            b.append("  auto-prefix blocks: ");
+            b.append(blockTreeMinItemsInAutoPrefix);
+            b.append(" - ");
+            b.append(blockTreeMaxItemsInAutoPrefix);
+            b.append("\n");
+          }
+        }
       } else {
         b.append("unset");
       }
@@ -575,6 +654,18 @@ public class FieldTypes {
         } else {
           b.append("\n  termVectors: unset");
         }
+        if (minTokenLength != null) {
+          b.append("\n  token length limit min=");
+          b.append(minTokenLength);
+          b.append(" max");
+          b.append(maxTokenLength);
+        }
+        if (maxTokenCount != null) {
+          b.append("\n  token count limit=");
+          b.append(maxTokenCount);
+          b.append(" consumeAllTokens=");
+          b.append(consumeAllTokens);
+        }
       }
       b.append('\n');
 
@@ -625,40 +716,49 @@ public class FieldTypes {
       // nocommit under codec control instead?
       out.writeString(name);
 
-      if (valueType == null) {
+      out.writeVInt(createdVersion.major);
+      out.writeVInt(createdVersion.minor);
+      out.writeVInt(createdVersion.bugfix);
+
+      switch (valueType) {
+      case NONE:
         out.writeByte((byte) 0);
-      } else {
-        switch (valueType) {
-        case TEXT:
-          out.writeByte((byte) 1);
-          break;
-        case SHORT_TEXT:
-          out.writeByte((byte) 2);
-          break;
-        case ATOM:
-          out.writeByte((byte) 3);
-          break;
-        case INT:
-          out.writeByte((byte) 4);
-          break;
-        case FLOAT:
-          out.writeByte((byte) 5);
-          break;
-        case LONG:
-          out.writeByte((byte) 6);
-          break;
-        case DOUBLE:
-          out.writeByte((byte) 7);
-          break;
-        case BINARY:
-          out.writeByte((byte) 8);
-          break;
-        case BOOLEAN:
-          out.writeByte((byte) 9);
-          break;
-        default:
-          throw new AssertionError("missing ValueType in switch");
-        }
+        break;
+      case TEXT:
+        out.writeByte((byte) 1);
+        break;
+      case SHORT_TEXT:
+        out.writeByte((byte) 2);
+        break;
+      case ATOM:
+        out.writeByte((byte) 3);
+        break;
+      case INT:
+        out.writeByte((byte) 4);
+        break;
+      case FLOAT:
+        out.writeByte((byte) 5);
+        break;
+      case LONG:
+        out.writeByte((byte) 6);
+        break;
+      case DOUBLE:
+        out.writeByte((byte) 7);
+        break;
+      case BINARY:
+        out.writeByte((byte) 8);
+        break;
+      case BOOLEAN:
+        out.writeByte((byte) 9);
+        break;
+      case DATE:
+        out.writeByte((byte) 10);
+        break;
+      case INET_ADDRESS:
+        out.writeByte((byte) 11);
+        break;
+      default:
+        throw new AssertionError("missing ValueType in switch");
       }
 
       if (docValuesTypeSet == false) {
@@ -695,6 +795,10 @@ public class FieldTypes {
       writeNullableInteger(out, blockTreeMaxItemsInAutoPrefix);
       writeNullableInteger(out, analyzerPositionGap);
       writeNullableInteger(out, analyzerOffsetGap);
+      writeNullableInteger(out, minTokenLength);
+      writeNullableInteger(out, maxTokenLength);
+      writeNullableInteger(out, maxTokenCount);
+      writeNullableBoolean(out, consumeAllTokens);
       writeNullableBoolean(out, stored);
       writeNullableBoolean(out, sortable);
       writeNullableBoolean(out, sortReversed);
@@ -800,10 +904,12 @@ public class FieldTypes {
     public FieldType(DataInput in) throws IOException {
       // nocommit under codec control instead?
       name = in.readString();
+      createdVersion = Version.fromBits(in.readVInt(), in.readVInt(), in.readVInt());
+
       byte b = in.readByte();
       switch (b) {
       case 0:
-        valueType = null;
+        valueType = ValueType.NONE;
         break;
       case 1:
         valueType = ValueType.TEXT;
@@ -832,6 +938,12 @@ public class FieldTypes {
       case 9:
         valueType = ValueType.BOOLEAN;
         break;
+      case 10:
+        valueType = ValueType.DATE;
+        break;
+      case 11:
+        valueType = ValueType.INET_ADDRESS;
+        break;
       default:
         throw new CorruptIndexException("invalid byte for ValueType: " + b, in);
       }
@@ -876,6 +988,10 @@ public class FieldTypes {
       blockTreeMaxItemsInAutoPrefix = readNullableInteger(in);
       analyzerPositionGap = readNullableInteger(in);
       analyzerOffsetGap = readNullableInteger(in);
+      minTokenLength = readNullableInteger(in);
+      maxTokenLength = readNullableInteger(in);
+      maxTokenCount = readNullableInteger(in);
+      consumeAllTokens = readNullableBoolean(in);
       stored = readNullableBoolean(in);
       sortable = readNullableBoolean(in);
       sortReversed = readNullableBoolean(in);
@@ -1199,10 +1315,11 @@ public class FieldTypes {
       protected Analyzer getWrappedAnalyzer(String fieldName) {
         FieldType field = fields.get(fieldName);
         if (field == null) {
+          // Must be lenient in case app is using low-schema API during indexing:
           return defaultIndexAnalyzer;
         }
-        if (field.indexAnalyzer != null) {
-          return field.indexAnalyzer;
+        if (field.wrappedIndexAnalyzer != null) {
+          return field.wrappedIndexAnalyzer;
         } else if (field.valueType == ValueType.ATOM) {
           // BUG
           illegalState(fieldName, "ATOM fields should not be analyzed during indexing");
@@ -1216,10 +1333,11 @@ public class FieldTypes {
       protected Analyzer getWrappedAnalyzer(String fieldName) {
         FieldType field = fields.get(fieldName);
         if (field == null) {
+          // Must be lenient in case app used low-schema API during indexing:
           return defaultQueryAnalyzer;
         }
-        if (field.queryAnalyzer != null) {
-          return field.queryAnalyzer;
+        if (field.wrappedQueryAnalyzer != null) {
+          return field.wrappedQueryAnalyzer;
         } else if (field.valueType == ValueType.ATOM) {
           return KEYWORD_ANALYZER;
         }
@@ -1284,6 +1402,7 @@ public class FieldTypes {
           current.indexAnalyzer = null;
         }
       }
+      current.reWrapAnalyzers(defaultIndexAnalyzer, defaultQueryAnalyzer);
       changed();
     } else {
       illegalState(fieldName, "analyzer was already set");
@@ -1314,6 +1433,7 @@ public class FieldTypes {
           current.queryAnalyzer = null;
         }
       }
+      current.reWrapAnalyzers(defaultIndexAnalyzer, defaultQueryAnalyzer);
       changed();
     } else {
       illegalState(fieldName, "analyzer was already set");
@@ -1373,6 +1493,95 @@ public class FieldTypes {
   public synchronized boolean getMultiValued(String fieldName) {
     return getFieldType(fieldName).multiValued == Boolean.TRUE;
   }
+  
+  /** Require that all tokens indexed for this field fall between the min and max
+   *  length, inclusive.  Any too-short or too-long tokens are silently discarded. */
+  public synchronized void setMinMaxTokenLength(String fieldName, int minTokenLength, int maxTokenLength) {
+    FieldType current = fields.get(fieldName);
+    if (current == null) {
+      current = new FieldType(fieldName);
+      current.minTokenLength = minTokenLength;
+      current.maxTokenLength = maxTokenLength;
+      fields.put(fieldName, current);
+      current.reWrapAnalyzers(defaultIndexAnalyzer, defaultQueryAnalyzer);
+      changed();
+    } else if (current.minTokenLength == null ||
+               current.minTokenLength.intValue() != minTokenLength ||
+               current.maxTokenLength.intValue() != maxTokenLength) {
+      Integer oldMin = current.minTokenLength;
+      Integer oldMax = current.maxTokenLength;
+      boolean success = false;
+      try {
+        current.minTokenLength = minTokenLength;
+        current.maxTokenLength = maxTokenLength;
+        current.validate();
+        success = true;
+      } finally {
+        if (success == false) {
+          current.minTokenLength = oldMin;
+          current.maxTokenLength = oldMax;
+        }
+      }
+      current.reWrapAnalyzers(defaultIndexAnalyzer, defaultQueryAnalyzer);
+      changed();
+    }
+  }
+
+  // nocommit clearMinMaxTokenLength
+
+  public synchronized Integer getMinTokenLength(String fieldName) {
+    return getFieldType(fieldName).minTokenLength;
+  }
+
+  public synchronized Integer getMaxTokenLength(String fieldName) {
+    return getFieldType(fieldName).maxTokenLength;
+  }
+
+  public synchronized void setMaxTokenCount(String fieldName, int maxTokenCount) {
+    setMaxTokenCount(fieldName, maxTokenCount, false);
+  }
+
+  // nocommit clearMaxTokenCount
+
+  /** Only index up to maxTokenCount tokens for this field. */
+  public synchronized void setMaxTokenCount(String fieldName, int maxTokenCount, boolean consumeAllTokens) {
+    FieldType current = fields.get(fieldName);
+    if (current == null) {
+      current = new FieldType(fieldName);
+      current.maxTokenCount = maxTokenCount;
+      current.consumeAllTokens = consumeAllTokens;
+      fields.put(fieldName, current);
+      current.reWrapAnalyzers(defaultIndexAnalyzer, defaultQueryAnalyzer);
+      changed();
+    } else if (current.maxTokenCount == null ||
+               current.maxTokenCount.intValue() != maxTokenCount ||
+               current.consumeAllTokens.booleanValue() != consumeAllTokens) {
+      Integer oldMax = current.maxTokenCount;
+      Boolean oldConsume = current.consumeAllTokens;
+      boolean success = false;
+      try {
+        current.maxTokenCount = maxTokenCount;
+        current.consumeAllTokens = consumeAllTokens;
+        current.validate();
+        success = true;
+      } finally {
+        if (success == false) {
+          current.maxTokenCount = maxTokenCount;
+          current.consumeAllTokens = consumeAllTokens;
+        }
+      }
+      current.reWrapAnalyzers(defaultIndexAnalyzer, defaultQueryAnalyzer);
+      changed();
+    }
+  }
+
+  public synchronized Integer getMaxTokenCount(String fieldName) {
+    return getFieldType(fieldName).maxTokenCount;
+  }
+
+  public synchronized Boolean getMaxTokenCountConsumeAllTokens(String fieldName) {
+    return getFieldType(fieldName).consumeAllTokens;
+  }
 
   /** The gap that should be added to token positions between each multi-valued field. */
   public synchronized void setAnalyzerPositionGap(String fieldName, int gap) {
@@ -1553,7 +1762,7 @@ public class FieldTypes {
       changed();
     } else if (current.sortable == Boolean.FALSE) {
       illegalState(fieldName, "sorting was already disabled");
-    } else if (current.sortReversed != reversed) {
+    } else if (current.sortReversed == null || current.sortReversed.booleanValue() != reversed) {
       current.sortReversed = reversed;
       changed();
     }
@@ -1920,7 +2129,7 @@ public class FieldTypes {
 
   /** Changes index options for this field.  This can be set to any
    *  value if it's not already set for the provided field; otherwise
-   *  it can only be downgraded as low as DOCS_ONLY but never unset
+   *  it can only be downgraded as low as DOCS but never unset
    *  entirely (once indexed, always indexed). */
   public synchronized void setIndexOptions(String fieldName, IndexOptions indexOptions) {
     ensureWritable();
@@ -1999,7 +2208,7 @@ public class FieldTypes {
       }
       changed();
     } else if (current.docValuesType != dvType) {
-      illegalState(fieldName, "cannot change from docValuesType " + current.docValuesType + " to docValutesType " + dvType);
+      illegalState(fieldName, "cannot change docValuesType from " + current.docValuesType + " to " + dvType);
     }
   }
 
@@ -2023,7 +2232,7 @@ public class FieldTypes {
       fields.put(fieldName, current);
       setDefaults(current);
       changed();
-    } else if (current.valueType == null) {
+    } else if (current.valueType == ValueType.NONE) {
       // This can happen if e.g. the app first calls FieldTypes.setStored(...)
       boolean success = false;
       try {
@@ -2032,7 +2241,7 @@ public class FieldTypes {
         success = true;
       } finally {
         if (success == false) {
-          current.valueType = null;
+          current.valueType = ValueType.NONE;
         }
       }
       setDefaults(current);
@@ -2057,7 +2266,7 @@ public class FieldTypes {
         current.indexOptions = IndexOptions.NONE;
       }
       changed();
-    } else if (current.valueType == null) {
+    } else if (current.valueType == ValueType.NONE) {
       // This can happen if e.g. the app first calls FieldTypes.setStored(...)
       boolean success = false;
       try {
@@ -2069,7 +2278,7 @@ public class FieldTypes {
         success = true;
       } finally {
         if (success == false) {
-          current.valueType = null;
+          current.valueType = ValueType.NONE;
         }
       }
       setDefaults(current);
@@ -2082,11 +2291,14 @@ public class FieldTypes {
   // ncommit move this method inside FildType:
   private void setDefaults(FieldType field) {
     switch (field.valueType) {
-
+    case NONE:
+      // bug
+      throw new AssertionError("valueType should not be NONE");
     case INT:
     case FLOAT:
     case LONG:
     case DOUBLE:
+    case DATE:
       if (field.highlighted == null) {
         field.highlighted = Boolean.FALSE;
       }
@@ -2179,6 +2391,7 @@ public class FieldTypes {
       break;
 
     case ATOM:
+    case INET_ADDRESS:
       if (field.highlighted == null) {
         field.highlighted = Boolean.FALSE;
       }
@@ -2482,6 +2695,23 @@ public class FieldTypes {
     return new TermQuery(new Term(fieldName, new BytesRef(value)));
   }
 
+  public Query newInetAddressTermQuery(String fieldName, InetAddress token) {
+    // Field must exist:
+    FieldType fieldType = getFieldType(fieldName);
+
+    // Field must be indexed:
+    if (fieldType.indexOptions == IndexOptions.NONE) {
+      illegalState(fieldName, "cannot create term query: this field was not indexed");
+    }
+
+    // Field must be InetAddress:
+    if (fieldType.valueType != ValueType.INET_ADDRESS) {
+      illegalState(fieldName, "inet address term query must have valueType INET_ADDRESS; got " + fieldType.valueType);
+    }
+
+    return new TermQuery(new Term(fieldName, new BytesRef(token.getAddress())));
+  }
+
   // nocommit shouldn't this be a filter?
   public Query newRangeQuery(String fieldName, Number min, boolean minInclusive, Number max, boolean maxInclusive) {
 
@@ -2555,6 +2785,59 @@ public class FieldTypes {
     return new TermRangeQuery(fieldName, minTerm, maxTerm, minInclusive, maxInclusive);
   }
 
+  // nocommit Date sugar for a range query matching a specific hour/day/month/year/etc.?  need locale/timezone... should we use DateTools?
+
+  // nocommit shouldn't this be a filter?
+  public Query newRangeQuery(String fieldName, Date min, boolean minInclusive, Date max, boolean maxInclusive) {
+
+    // Field must exist:
+    FieldType fieldType = getFieldType(fieldName);
+
+    // Field must be indexed:
+    if (fieldType.indexOptions == IndexOptions.NONE) {
+      illegalState(fieldName, "cannot create range query: this field was not indexed");
+    }
+
+    if (fieldType.valueType != ValueType.DATE) {
+      illegalState(fieldName, "cannot create range query: expected valueType=DATE but got: " + fieldType.valueType);
+    }
+
+    if (fieldType.fastRanges != Boolean.TRUE) {
+      illegalState(fieldName, "this field was not indexed for fast ranges");
+    }
+
+    BytesRef minTerm = min == null ? null : Document2.longToBytes(min.getTime());
+    BytesRef maxTerm = max == null ? null : Document2.longToBytes(max.getTime());
+
+    return new TermRangeQuery(fieldName, minTerm, maxTerm, minInclusive, maxInclusive);
+  }
+
+  // nocommit shouldn't this be a filter?
+  // nocommit also add "range filter using net mask" sugar version
+  public Query newRangeQuery(String fieldName, InetAddress min, boolean minInclusive, InetAddress max, boolean maxInclusive) {
+
+    // Field must exist:
+    FieldType fieldType = getFieldType(fieldName);
+
+    // Field must be indexed:
+    if (fieldType.indexOptions == IndexOptions.NONE) {
+      illegalState(fieldName, "cannot create range query: this field was not indexed");
+    }
+
+    if (fieldType.valueType != ValueType.INET_ADDRESS) {
+      illegalState(fieldName, "cannot create range query: expected valueType=INET_ADDRESS but got: " + fieldType.valueType);
+    }
+
+    if (fieldType.fastRanges != Boolean.TRUE) {
+      illegalState(fieldName, "this field was not indexed for fast ranges");
+    }
+
+    BytesRef minTerm = min == null ? null : new BytesRef(min.getAddress());
+    BytesRef maxTerm = max == null ? null : new BytesRef(max.getAddress());
+
+    return new TermRangeQuery(fieldName, minTerm, maxTerm, minInclusive, maxInclusive);
+  }
+
   // nocommit newPhraseQuery?
 
   /** Builds a sort from arbitrary list of fieldName, reversed pairs. */
@@ -2652,8 +2935,9 @@ public class FieldTypes {
         }
         return sortField;
       }
-      
+
     case LONG:
+    case DATE:
       {
         SortField sortField;
         if (fieldType.multiValued == Boolean.TRUE) {
@@ -2707,6 +2991,7 @@ public class FieldTypes {
     case ATOM:
     case BINARY:
     case BOOLEAN:
+    case INET_ADDRESS:
       SortField sortField;
       {
         if (fieldType.multiValued == Boolean.TRUE) {
@@ -2771,9 +3056,9 @@ public class FieldTypes {
     RAMOutputStream out = new RAMOutputStream(file, true);
     CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT);
     
-    out.writeInt(indexCreatedVersion.major);
-    out.writeInt(indexCreatedVersion.minor);
-    out.writeInt(indexCreatedVersion.bugfix);
+    out.writeVInt(indexCreatedVersion.major);
+    out.writeVInt(indexCreatedVersion.minor);
+    out.writeVInt(indexCreatedVersion.bugfix);
 
     out.writeVInt(fields.size());
     for(FieldType fieldType : fields.values()) {
@@ -2801,7 +3086,7 @@ public class FieldTypes {
 
     CodecUtil.checkHeader(in, CODEC_NAME, VERSION_START, VERSION_START);
 
-    Version indexCreatedVersion = Version.fromBits(in.readInt(), in.readInt(), in.readInt());
+    Version indexCreatedVersion = Version.fromBits(in.readVInt(), in.readVInt(), in.readVInt());
 
     int count = in.readVInt();
     for(int i=0;i<count;i++) {

Copied: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/FilteringTokenFilter.java (from r1635912, lucene/dev/branches/lucene6005/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/FilteringTokenFilter.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/FilteringTokenFilter.java?p2=lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/FilteringTokenFilter.java&p1=lucene/dev/branches/lucene6005/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/FilteringTokenFilter.java&r1=1635912&r2=1636293&rev=1636293&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/FilteringTokenFilter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/FilteringTokenFilter.java Mon Nov  3 09:22:24 2014
@@ -1,4 +1,4 @@
-package org.apache.lucene.analysis.util;
+package org.apache.lucene.document;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -23,13 +23,15 @@ import org.apache.lucene.analysis.TokenF
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 
+// nocommit forked from analysis module
+
 /**
  * Abstract base class for TokenFilters that may remove tokens.
  * You have to implement {@link #accept} and return a boolean if the current
  * token should be preserved. {@link #incrementToken} uses this method
  * to decide if a token should be passed to the caller.
  */
-public abstract class FilteringTokenFilter extends TokenFilter {
+abstract class FilteringTokenFilter extends TokenFilter {
 
   private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
   private int skippedPositions;

Copied: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/LengthFilter.java (from r1635912, lucene/dev/branches/lucene6005/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LengthFilter.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/LengthFilter.java?p2=lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/LengthFilter.java&p1=lucene/dev/branches/lucene6005/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LengthFilter.java&r1=1635912&r2=1636293&rev=1636293&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LengthFilter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/LengthFilter.java Mon Nov  3 09:22:24 2014
@@ -1,4 +1,4 @@
-package org.apache.lucene.analysis.miscellaneous;
+package org.apache.lucene.document;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -17,8 +17,9 @@ package org.apache.lucene.analysis.misce
  * limitations under the License.
  */
 
+// nocommit forked from analysis module
+
 import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.util.FilteringTokenFilter;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 
 /**
@@ -27,7 +28,7 @@ import org.apache.lucene.analysis.tokena
  * Note: Length is calculated as the number of UTF-16 code units.
  * </p>
  */
-public final class LengthFilter extends FilteringTokenFilter {
+final class LengthFilter extends FilteringTokenFilter {
 
   private final int min;
   private final int max;

Added: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/LimitTokenCountFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/LimitTokenCountFilter.java?rev=1636293&view=auto
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/LimitTokenCountFilter.java (added)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/LimitTokenCountFilter.java Mon Nov  3 09:22:24 2014
@@ -0,0 +1,97 @@
+package org.apache.lucene.document;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.analysis.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+
+// nocommit forked from analysis module
+
+/**
+ * This TokenFilter limits the number of tokens while indexing. It is
+ * a replacement for the maximum field length setting inside {@link org.apache.lucene.index.IndexWriter}.
+ * <p>
+ * By default, this filter ignores any tokens in the wrapped {@code TokenStream}
+ * once the limit has been reached, which can result in {@code reset()} being 
+ * called prior to {@code incrementToken()} returning {@code false}.  For most 
+ * {@code TokenStream} implementations this should be acceptable, and faster 
+ * then consuming the full stream. If you are wrapping a {@code TokenStream} 
+ * which requires that the full stream of tokens be exhausted in order to 
+ * function properly, use the 
+ * {@link #LimitTokenCountFilter(TokenStream,int,boolean) consumeAllTokens} 
+ * option.
+ */
+final class LimitTokenCountFilter extends TokenFilter {
+
+  private final int maxTokenCount;
+  private final boolean consumeAllTokens;
+  private int tokenCount = 0;
+  private boolean exhausted = false;
+
+  /**
+   * Build a filter that only accepts tokens up to a maximum number.
+   * This filter will not consume any tokens beyond the maxTokenCount limit
+   *
+   * @see #LimitTokenCountFilter(TokenStream,int,boolean)
+   */
+  public LimitTokenCountFilter(TokenStream in, int maxTokenCount) {
+    this(in, maxTokenCount, false);
+  }
+
+  /**
+   * Build an filter that limits the maximum number of tokens per field.
+   * @param in the stream to wrap
+   * @param maxTokenCount max number of tokens to produce
+   * @param consumeAllTokens whether all tokens from the input must be consumed even if maxTokenCount is reached.
+   */
+  public LimitTokenCountFilter(TokenStream in, int maxTokenCount, boolean consumeAllTokens) {
+    super(in);
+    if (maxTokenCount < 1) {
+      throw new IllegalArgumentException("maxTokenCount must be greater than zero");
+    }
+    this.maxTokenCount = maxTokenCount;
+    this.consumeAllTokens = consumeAllTokens;
+  }
+  
+  @Override
+  public boolean incrementToken() throws IOException {
+    if (exhausted) {
+      return false;
+    } else if (tokenCount < maxTokenCount) {
+      if (input.incrementToken()) {
+        tokenCount++;
+        return true;
+      } else {
+        exhausted = true;
+        return false;
+      }
+    } else {
+      while (consumeAllTokens && input.incrementToken()) { /* NOOP */ }
+      return false;
+    }
+  }
+
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    tokenCount = 0;
+    exhausted = false;
+  }
+}

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ParallelCompositeReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ParallelCompositeReader.java?rev=1636293&r1=1636292&r2=1636293&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ParallelCompositeReader.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ParallelCompositeReader.java Mon Nov  3 09:22:24 2014
@@ -23,6 +23,8 @@ import java.util.IdentityHashMap;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.lucene.document.FieldTypes;
+
 /** An {@link CompositeReader} which reads multiple, parallel indexes.  Each index added
  * must have the same number of documents, and exactly the same hierarchical subreader structure,
  * but typically each contains different fields. Deletions are taken from the first reader.
@@ -48,6 +50,7 @@ import java.util.Set;
  */
 public class ParallelCompositeReader extends BaseCompositeReader<IndexReader> {
   private final boolean closeSubReaders;
+  private final FieldTypes fieldTypes;
   private final Set<IndexReader> completeReaderSet =
     Collections.newSetFromMap(new IdentityHashMap<IndexReader,Boolean>());
 
@@ -77,10 +80,18 @@ public class ParallelCompositeReader ext
         reader.incRef();
       }
     }
+    // nocommit must check that they are congruent:
+    fieldTypes = readers.length > 0 ? readers[0].getFieldTypes() : null;
+
     // finally add our own synthetic readers, so we close or decRef them, too (it does not matter what we do)
     completeReaderSet.addAll(getSequentialSubReaders());
   }
 
+  @Override
+  public FieldTypes getFieldTypes() {
+    return fieldTypes;
+  }
+
   private static IndexReader[] prepareSubReaders(CompositeReader[] readers, CompositeReader[] storedFieldsReaders) throws IOException {
     if (readers.length == 0) {
       if (storedFieldsReaders.length > 0)

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java?rev=1636293&r1=1636292&r2=1636293&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java Mon Nov  3 09:22:24 2014
@@ -26,6 +26,7 @@ import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
+import org.apache.lucene.document.FieldTypes;
 import org.apache.lucene.util.Bits;
 
 
@@ -58,7 +59,8 @@ public class ParallelLeafReader extends 
   private final boolean hasDeletions;
   private final SortedMap<String,LeafReader> fieldToReader = new TreeMap<>();
   private final SortedMap<String,LeafReader> tvFieldToReader = new TreeMap<>();
-  
+  private final FieldTypes fieldTypes;
+
   /** Create a ParallelLeafReader based on the provided
    *  readers; auto-closes the given readers on {@link #close()}. */
   public ParallelLeafReader(LeafReader... readers) throws IOException {
@@ -85,9 +87,12 @@ public class ParallelLeafReader extends 
       this.maxDoc = first.maxDoc();
       this.numDocs = first.numDocs();
       this.hasDeletions = first.hasDeletions();
+      // nocommit must verify field types are congruent and take union?
+      this.fieldTypes = first.getFieldTypes();
     } else {
       this.maxDoc = this.numDocs = 0;
       this.hasDeletions = false;
+      this.fieldTypes = null;
     }
     Collections.addAll(completeReaderSet, this.parallelReaders);
     Collections.addAll(completeReaderSet, this.storedFieldsReaders);
@@ -140,6 +145,11 @@ public class ParallelLeafReader extends 
   }
 
   @Override
+  public FieldTypes getFieldTypes() {
+    return fieldTypes;
+  }
+
+  @Override
   public String toString() {
     final StringBuilder buffer = new StringBuilder("ParallelLeafReader(");
     for (final Iterator<LeafReader> iter = completeReaderSet.iterator(); iter.hasNext();) {

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java?rev=1636293&r1=1636292&r2=1636293&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java Mon Nov  3 09:22:24 2014
@@ -21,11 +21,11 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.lucene.util.Bits;
-
+import org.apache.lucene.document.FieldTypes;
 import org.apache.lucene.index.MultiDocValues.MultiSortedDocValues;
 import org.apache.lucene.index.MultiDocValues.MultiSortedSetDocValues;
 import org.apache.lucene.index.MultiDocValues.OrdinalMap;
+import org.apache.lucene.util.Bits;
 
 /**
  * This class forces a composite reader (eg a {@link
@@ -47,6 +47,7 @@ public final class SlowCompositeReaderWr
   private final CompositeReader in;
   private final Fields fields;
   private final Bits liveDocs;
+  private final FieldTypes fieldTypes;
   
   /** This method is sugar for getting an {@link LeafReader} from
    * an {@link IndexReader} of any kind. If the reader is already atomic,
@@ -67,6 +68,12 @@ public final class SlowCompositeReaderWr
     fields = MultiFields.getFields(in);
     liveDocs = MultiFields.getLiveDocs(in);
     in.registerParentReader(this);
+    fieldTypes = in.getFieldTypes();
+  }
+
+  @Override
+  public FieldTypes getFieldTypes() {
+    return fieldTypes;
   }
 
   @Override

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=1636293&r1=1636292&r2=1636293&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 Mon Nov  3 09:22:24 2014
@@ -18,6 +18,11 @@ package org.apache.lucene.document;
  */
 
 import java.io.StringReader;
+import java.net.InetAddress;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Locale;
+import java.util.TimeZone;
 
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
 import org.apache.lucene.analysis.CannedTokenStream;
@@ -1143,8 +1148,8 @@ public class TestDocument2 extends Lucen
     TopDocs hits = s.search(new MatchAllDocsQuery(), 3, fieldTypes.newSort("int"));
     assertEquals(3, hits.totalHits);
     assertEquals("1", s.doc(hits.scoreDocs[0].doc).getString("id"));
-    assertEquals("2", s.doc(hits.scoreDocs[1].doc).getString("id"));
-    assertEquals("0", s.doc(hits.scoreDocs[2].doc).getString("id"));
+    assertEquals("0", s.doc(hits.scoreDocs[1].doc).getString("id"));
+    assertEquals("2", s.doc(hits.scoreDocs[2].doc).getString("id"));
     r.close();
     w.close();
     dir.close();
@@ -1176,8 +1181,8 @@ public class TestDocument2 extends Lucen
     TopDocs hits = s.search(new MatchAllDocsQuery(), 3, fieldTypes.newSort("int", true));
     assertEquals(3, hits.totalHits);
     assertEquals("0", s.doc(hits.scoreDocs[0].doc).getString("id"));
-    assertEquals("2", s.doc(hits.scoreDocs[1].doc).getString("id"));
-    assertEquals("1", s.doc(hits.scoreDocs[2].doc).getString("id"));
+    assertEquals("1", s.doc(hits.scoreDocs[1].doc).getString("id"));
+    assertEquals("2", s.doc(hits.scoreDocs[2].doc).getString("id"));
     r.close();
     w.close();
     dir.close();
@@ -1354,9 +1359,9 @@ public class TestDocument2 extends Lucen
     IndexSearcher s = newSearcher(r);
     TopDocs hits = s.search(new MatchAllDocsQuery(), 3, fieldTypes.newSort("atom"));
     assertEquals(3, hits.totalHits);
-    assertEquals("2", s.doc(hits.scoreDocs[0].doc).getString("id"));
-    assertEquals("1", s.doc(hits.scoreDocs[1].doc).getString("id"));
-    assertEquals("0", s.doc(hits.scoreDocs[2].doc).getString("id"));
+    assertEquals("1", s.doc(hits.scoreDocs[0].doc).getString("id"));
+    assertEquals("0", s.doc(hits.scoreDocs[1].doc).getString("id"));
+    assertEquals("2", s.doc(hits.scoreDocs[2].doc).getString("id"));
     r.close();
     w.close();
     dir.close();
@@ -1395,6 +1400,234 @@ public class TestDocument2 extends Lucen
     dir.close();
   }
 
+  public void testMinMaxTokenLength() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
+    w.getFieldTypes().setMinMaxTokenLength("field", 2, 7);
+    w.commit();
+
+    Document2 doc = w.newDocument();
+    doc.addLargeText("field", "hello a toobigterm");
+    w.addDocument(doc);
+
+    DirectoryReader r = DirectoryReader.open(w, true);
+    FieldTypes fieldTypes = r.getFieldTypes();
+    assertEquals(2, fieldTypes.getMinTokenLength("field").intValue());
+    assertEquals(7, fieldTypes.getMaxTokenLength("field").intValue());
+
+    IndexSearcher s = newSearcher(r);
+    assertEquals(1, s.search(fieldTypes.newStringTermQuery("field", "hello"), 1).totalHits);
+    assertEquals(0, s.search(fieldTypes.newStringTermQuery("field", "a"), 1).totalHits);
+    assertEquals(0, s.search(fieldTypes.newStringTermQuery("field", "toobigterm"),1 ).totalHits);
+    r.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testMaxTokenCount() throws Exception {
+    Directory dir = newDirectory();
+    MockAnalyzer a = new MockAnalyzer(random());
+    // MockAnalyzer is angry that we don't consume all tokens:
+    a.setEnableChecks(false);
+    IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(a));
+    FieldTypes fieldTypes = w.getFieldTypes();
+    fieldTypes.setMaxTokenCount("field", 3);
+    w.commit();
+
+    Document2 doc = w.newDocument();
+    doc.addLargeText("field", "hello a toobigterm goodbye");
+    w.addDocument(doc);
+
+    DirectoryReader r = DirectoryReader.open(w, true);
+    fieldTypes = r.getFieldTypes();
+    assertEquals(3, fieldTypes.getMaxTokenCount("field").intValue());
+
+    IndexSearcher s = newSearcher(r);
+    assertEquals(1, s.search(fieldTypes.newStringTermQuery("field", "hello"), 1).totalHits);
+    assertEquals(1, s.search(fieldTypes.newStringTermQuery("field", "a"), 1).totalHits);
+    assertEquals(1, s.search(fieldTypes.newStringTermQuery("field", "toobigterm"), 1).totalHits);
+    assertEquals(0, s.search(fieldTypes.newStringTermQuery("field", "goodbye"), 1).totalHits);
+    r.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testMaxTokenCountConsumeAll() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
+    FieldTypes fieldTypes = w.getFieldTypes();
+    fieldTypes.setMaxTokenCount("field", 3, true);
+
+    Document2 doc = w.newDocument();
+    doc.addLargeText("field", "hello a toobigterm goodbye");
+    w.addDocument(doc);
+
+    DirectoryReader r = DirectoryReader.open(w, true);
+    fieldTypes = r.getFieldTypes();
+    assertEquals(3, fieldTypes.getMaxTokenCount("field").intValue());
+
+    IndexSearcher s = newSearcher(r);
+    assertEquals(1, s.search(fieldTypes.newStringTermQuery("field", "hello"), 1).totalHits);
+    assertEquals(1, s.search(fieldTypes.newStringTermQuery("field", "a"), 1).totalHits);
+    assertEquals(1, s.search(fieldTypes.newStringTermQuery("field", "toobigterm"), 1).totalHits);
+    assertEquals(0, s.search(fieldTypes.newStringTermQuery("field", "goodbye"), 1).totalHits);
+    r.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testExcSuddenlyEnableDocValues() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
+    FieldTypes fieldTypes = w.getFieldTypes();
+    fieldTypes.setDocValuesType("field", DocValuesType.NONE);
+
+    Document2 doc = w.newDocument();
+    doc.addInt("field", 17);
+    w.addDocument(doc);
+
+    try {
+      fieldTypes.setDocValuesType("field", DocValuesType.NUMERIC);
+      fail("did not hit exception");
+    } catch (IllegalStateException ise) {
+      assertEquals("field \"field\": cannot change docValuesType from NONE to NUMERIC", ise.getMessage());
+    }
+    w.close();
+    dir.close();
+  }
+
+  public void testDateSort() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
+
+    SimpleDateFormat parser = new SimpleDateFormat("MM/dd/yyyy", Locale.ROOT);
+    parser.setTimeZone(TimeZone.getTimeZone("GMT"));
+
+    Document2 doc = w.newDocument();
+    Date date0 = parser.parse("10/22/2014");
+    doc.addDate("date", date0);
+    doc.addAtom("id", "0");
+    w.addDocument(doc);
+
+    doc = w.newDocument();
+    Date date1 = parser.parse("10/21/2015");
+    doc.addDate("date", date1);
+    doc.addAtom("id", "1");
+    w.addDocument(doc);
+
+    w.getFieldTypes().enableSorting("date", true);
+
+    DirectoryReader r = DirectoryReader.open(w, true);
+    FieldTypes fieldTypes = r.getFieldTypes();
+
+    IndexSearcher s = newSearcher(r);
+    TopDocs hits = s.search(new MatchAllDocsQuery(), 10, fieldTypes.newSort("date"));
+    assertEquals(2, hits.totalHits);
+    Document2 hit = s.doc(hits.scoreDocs[0].doc);
+    assertEquals("1", hit.getString("id"));
+    assertEquals(date1, hit.getDate("date"));
+    hit = s.doc(hits.scoreDocs[1].doc);
+    assertEquals("0", hit.getString("id"));
+    assertEquals(date0, hit.getDate("date"));
+    r.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testDateRangeFilter() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
+
+    SimpleDateFormat parser = new SimpleDateFormat("MM/dd/yyyy", Locale.ROOT);
+    parser.setTimeZone(TimeZone.getTimeZone("GMT"));
+
+    Document2 doc = w.newDocument();
+    Date date0 = parser.parse("10/22/2014");
+    doc.addDate("date", date0);
+    doc.addAtom("id", "0");
+    w.addDocument(doc);
+
+    doc = w.newDocument();
+    Date date1 = parser.parse("10/21/2015");
+    doc.addDate("date", date1);
+    doc.addAtom("id", "1");
+    w.addDocument(doc);
+
+    DirectoryReader r = DirectoryReader.open(w, true);
+    FieldTypes fieldTypes = r.getFieldTypes();
+
+    IndexSearcher s = newSearcher(r);
+    assertEquals(2, s.search(fieldTypes.newRangeQuery("date", date0, true, date1, true), 1).totalHits);
+    assertEquals(1, s.search(fieldTypes.newRangeQuery("date", date0, true, date1, false), 1).totalHits);
+    assertEquals(0, s.search(fieldTypes.newRangeQuery("date", date0, false, date1, false), 1).totalHits);
+    assertEquals(1, s.search(fieldTypes.newRangeQuery("date", parser.parse("10/21/2014"), false, parser.parse("10/23/2014"), false), 1).totalHits);
+    r.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testInetAddressSort() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
+
+    Document2 doc = w.newDocument();
+    InetAddress inet0 = InetAddress.getByName("10.17.4.10");
+    doc.addInetAddress("inet", inet0);
+    doc.addAtom("id", "0");
+    w.addDocument(doc);
+
+    doc = w.newDocument();
+    InetAddress inet1 = InetAddress.getByName("10.17.4.22");
+    doc.addInetAddress("inet", inet1);
+    doc.addAtom("id", "1");
+    w.addDocument(doc);
+
+    DirectoryReader r = DirectoryReader.open(w, true);
+    FieldTypes fieldTypes = r.getFieldTypes();
+
+    IndexSearcher s = newSearcher(r);
+    TopDocs hits = s.search(new MatchAllDocsQuery(), 10, fieldTypes.newSort("inet"));
+    assertEquals(2, hits.totalHits);
+    Document2 hit = s.doc(hits.scoreDocs[0].doc);
+    assertEquals("0", hit.getString("id"));
+    assertEquals(inet0, hit.getInetAddress("inet"));
+    hit = s.doc(hits.scoreDocs[1].doc);
+    assertEquals("1", hit.getString("id"));
+    assertEquals(inet1, hit.getInetAddress("inet"));
+    r.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testInetAddressRangeFilter() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
+
+    Document2 doc = w.newDocument();
+    InetAddress inet0 = InetAddress.getByName("10.17.4.10");
+    doc.addInetAddress("inet", inet0);
+    doc.addAtom("id", "0");
+    w.addDocument(doc);
+
+    doc = w.newDocument();
+    InetAddress inet1 = InetAddress.getByName("10.17.4.22");
+    doc.addInetAddress("inet", inet1);
+    doc.addAtom("id", "1");
+    w.addDocument(doc);
+
+    DirectoryReader r = DirectoryReader.open(w, true);
+    FieldTypes fieldTypes = r.getFieldTypes();
+
+    IndexSearcher s = newSearcher(r);
+    assertEquals(2, s.search(fieldTypes.newRangeQuery("inet", inet0, true, inet1, true), 1).totalHits);
+    assertEquals(1, s.search(fieldTypes.newRangeQuery("inet", inet0, true, inet1, false), 1).totalHits);
+    assertEquals(0, s.search(fieldTypes.newRangeQuery("inet", inet0, false, inet1, false), 1).totalHits);
+    assertEquals(1, s.search(fieldTypes.newRangeQuery("inet", InetAddress.getByName("10.17.0.0"), true, InetAddress.getByName("10.17.4.20"), false), 1).totalHits);
+    r.close();
+    w.close();
+    dir.close();
+  }
+
   // nocommit test per-field analyzers
 
   // nocommit test per-field sims