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 2015/01/02 13:02:33 UTC

svn commit: r1649007 [2/10] - in /lucene/dev/branches/lucene6005/lucene: analysis/common/src/test/org/apache/lucene/analysis/core/ analysis/common/src/test/org/apache/lucene/collation/ benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ benchm...

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=1649007&r1=1649006&r2=1649007&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 Fri Jan  2 12:02:31 2015
@@ -1,5 +1,4 @@
 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
@@ -18,6 +17,7 @@ package org.apache.lucene.document;
  */
 
 import java.io.IOException;
+import java.lang.reflect.Constructor;
 import java.math.BigInteger;
 import java.net.InetAddress;
 import java.nio.charset.StandardCharsets;
@@ -48,20 +48,28 @@ import org.apache.lucene.index.IndexOpti
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexableFieldType;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.DocValuesRangeFilter;
 import org.apache.lucene.search.FieldComparator;
 import org.apache.lucene.search.FieldComparatorSource;
 import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.HalfFloatComparator;
 import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.LeftZeroPadTermRangeFilter;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.QueryWrapperFilter;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.SortedNumericSelector;
 import org.apache.lucene.search.SortedNumericSortField;
+import org.apache.lucene.search.SortedSetSelector;
 import org.apache.lucene.search.SortedSetSortField;
+import org.apache.lucene.search.TermFilter;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TermRangeFilter;
 import org.apache.lucene.search.similarities.PerFieldSimilarityWrapper;
@@ -81,171 +89,116 @@ import org.apache.lucene.util.NumericUti
 import org.apache.lucene.util.Version;
 
 // TODO
+//   - nocommit: back compat to old numeric fields
+//   - write/read of field types should be up to codec?
+//   - how should Codec integrate
+//   - analyzers are not serializable
 //   - 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?
-//   - can we simplify how "low level schema" (FieldInfo) merges itself?
-//     - only if this new API is the only way to add docs to IndexWriter ... hmm
 //   - CheckIndex could optionally do more validation
-//     - e.g. that the terms in a numeric field are correct (lead w/ prefix, 7 bit clean)
 //     - or that terms in an ipv6 field always have 16 bytes
 //   - add query time integration
 //     - query parsers
 //       - exc if invalid field name asked for
 //       - numeric range queries "just work"
-//     - creating queries, catching invalid field names, no positions indexed, etc.
-//     - prox queries can verify field was indexed w/ positions
-
-// tie into query parser
-//   default operators?
-//   default search field
+//       - default search field
 
-// tie into highlighter
-// tie into faceting
-// tie into index sorting
+// docsWithField and fieldExistsFilter are redundant if field is dv'd and indexed
 
-// nocommit should we just default to StandardAnalyzer
 
-// nocommit it's ridiculous to have the "sort missing as 0" option/default for numerics
-
-// nocommit how to allow extending this?
+// TODO: how to allow extending this?
 //   - geo
 //   - expressions
-//   - facets (FacetsConfig)
+//   - index sorting
+//   - suggesters
+//   - icu
+//   - facets (FacetsConfig, hierarchical fields)
 //   - doc blocks (nested) / joins
 
-// fold in compressing stored fields format params...how
-
-// nocommit icu collation field is difficult now
-
-// nocommit sugar API to retrieve values from DVs or stored fields or whatever?
-
 // nocommit how will future back-compat work?  segment must store field types as of when it was written?
 
-// nocommit PH should take this and validate highlighting was enabled?
-
 // nocommit run all monster tests
 
-// nocommit iterator over all fields / types?
-
-// nocommit allow adding array of atom values?  fieldnamesfield would use it?
-
-// nocommit optimize field exists filter to MatchAllBits when all docs in the seg have the field; same opto as range query when min < terms.min & max > terms.max
-
-// nocommit should exists filter use docsWithField?
-
-// nocommit xlog is easier w/ schema?
-
-// nocommit use better pf when field is unique
-
-// nocommit filter caching?  parent docs filter?
-
-// nocommit we could track here which fields are actually searched/filtered on ... and e.g. make use of this during warmers ...
-
-// nocommit do we allow mixing of binary and non-binary atom?
-
-// nocommit fix simple qp to optionally take this?
-
-// nocommit optimize term range query when it's really "matches all docs"?
-
-// nocommit but how can we randomized IWC for tests?
-
-// nocommit maybe we should filter out our key from commit user data?  scary if apps can ... mess with it accidentally?  or just store it
-// directly in SIS or someplace else
-
 // nocommit move to oal.index?
 
 // NO
+//   - filter caching?  parent blockdocs filter?
+//   - required
+//   - not null
+//   - "all" field
+//   - "copy" field
 //   - dynamic fields
 //   - can we somehow always store a "source"?  can be handled above
 //   - default value (value used if the field is null/missing): this seems silly wasteful, and layer above can handle it
 //   - sort proxy field ("when I sort by X you should actually sort by Y"): can be handled above
-
-// nocommit highlight proxy field (LUCENE-6061)
-
+//   - can/should we validate field names here?
 
 // LATER
+//   - time intervals
+//   - fold in compressing stored fields format params...how
+//   - add array of atom values?  fieldnamesfield would use it?
+//   - sugar for more/less ranges (where one end is null)
+//   - separate analyzer for phrase queries in suggesters
+//   - more sugar for common analyzers, e.g. accent folding (to ascii), case folding, ngrams for partial matching
+//   - can we somehow detect at search time if the field types you are using doesn't match the searcher you are now searching against?
+//   - add "partial match" option, that turns on ngrams?
+//   - can we move multi-field-ness out of IW?  so IW only gets a single instance of each field
+//   - use better pf when field is unique
+//   - nested/parent/child docs?
+//   - highlight proxy field (LUCENE-6061)
+//   - sugar API to retrieve values from DVs or stored fields or whatever?
+//   - we could track here which fields are actually searched/filtered on ... and e.g. make use of this during warmers ...
+//   - BigDecimal?
+//   - index-time sorting should be here too
+//   - make LiveFieldValues easier to use?
 //   - can we have test infra that randomly reopens writer?
 //   - newTermFilter?
+//   - PH should take this and validate highlighting was enabled?  (it already checks for OFFSETS in postings)
 
-// nocommit live values?
-
-// nocommit default qp operator
-
-// nocommit copy 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?
-
-// nocommit back compat: how to handle pre-schema indices
-
-// nocommit maybe have a settable global default for "stored"?
-
-// nocommit can/should we validate field names here?
-
-// nocommit make ValueType public?  add setter so you can set that too?
-
-// language for the field?  (to default collator)
+// nocommit -- can we sort on a field that ran through analysis (and made a single token), e.g. case / accent folding
 
 // nocommit sort order, sort options (e.g. for collation)
 //   case sensitive, punc sensitive, accent sensitive
-//   can we fold in ICUCollationDocValuesField somehow...
-
-// nocommit suggesters
-
-// nocommit index-time sorting should be here too
-
-// nocommit can we require use of analyzer factories?
 
 // nocommit accent removal and lowercasing for wildcards should just work
 
-// separate analyzer for phrase queries in suggesters
-
-// nocommit Index class? xlog?
-
-// nocommit how to randomize IWC?  RIW?
-
 // nocommit fix all change methods to call validate / rollback
 
-// nocommit can we move multi-field-ness out of IW?  so IW only gets a single instance of each field
-
-// nocommit should mulit-valued fields be added as straight array?
-
-// nocommit nested/parent/child docs?
-
-// nocommit "all" field:
+// nocommit controlling compression of stored fields, norms
 
-// nocommit doc blocks?
+/** Records how each field is indexed, stored, etc.
+ *
+ * @lucene.experimental */
 
-// nocomit hierarchical fields?
+public class FieldTypes {
 
-// nocommit required?  not null?
+  public static final int DEFAULT_POSITION_GAP = 0;
 
-// nocommit BigDecimal?
+  public static final int DEFAULT_OFFSET_GAP = 1;
 
-/** Records how each field is indexed, stored, etc.  This class persists
- *  its state using {@link IndexWriter#setCommitData}, using the
- *  {@link FieldTypes#FIELD_PROPERTIES_KEY} key. */
+  /** Key used to store the field types inside {@link IndexWriter#setCommitData}. */
+  public static final String FIELD_TYPES_KEY = "FieldTypes";
+  
+  public static final String FIELD_NAMES_FIELD = "$fieldnames";
 
-public class FieldTypes {
+  public static final int VERSION_START = 0;
 
-  public static final int DEFAULT_POSITION_GAP = 0;
+  public static final int VERSION_CURRENT = VERSION_START;
 
-  public static final int DEFAULT_OFFSET_GAP = 1;
+  public static final String CODEC_NAME = "FieldTypes";
 
   public enum ValueType {
     NONE,
     TEXT,
     SHORT_TEXT,
-    ATOM,  // nocommit binary sort of overlaps w/ this?
+    ATOM,
     INT,
     HALF_FLOAT,
     FLOAT,
     LONG,
     DOUBLE,
     BIG_INT,
-    BINARY, // nocommit rename to bytes?
+    BINARY,
     BOOLEAN,
     DATE,
     INET_ADDRESS,
@@ -255,8 +208,6 @@ public class FieldTypes {
 
   private final boolean readOnly;
 
-  public static final String FIELD_NAMES_FIELD = "$fieldnames";
-
   /** So exists filters are fast */
   boolean enableExistsFilters = true;
   private boolean indexedDocs;
@@ -269,9 +220,9 @@ public class FieldTypes {
   private final Analyzer defaultIndexAnalyzer;
 
   // Null when we are not readOnly:
-  private final Analyzer defaultQueryAnalyzer;
+  private Analyzer defaultQueryAnalyzer;
 
-  private final Similarity defaultSimilarity;
+  private Similarity defaultSimilarity;
 
   /** Used only in memory to record when something changed. */
   private long changeCount;
@@ -285,56 +236,6 @@ public class FieldTypes {
     // 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;
-    }
-
-    /** Copy constructor. */
-    FieldType(FieldType other) {
-      // nocommit how to make sure nothing is missing here?  can/should we just use default clone impl?
-      this.name = other.name;
-      this.createdVersion = other.createdVersion;
-      this.valueType = other.valueType;
-      this.docValuesType = other.docValuesType;
-      this.docValuesTypeSet = other.docValuesTypeSet;
-      this.isUnique = other.isUnique;
-      this.blockTreeMinItemsInBlock = other.blockTreeMinItemsInBlock;
-      this.blockTreeMaxItemsInBlock = other.blockTreeMaxItemsInBlock;
-      this.blockTreeMinItemsInAutoPrefix = other.blockTreeMinItemsInAutoPrefix;
-      this.blockTreeMaxItemsInAutoPrefix = other.blockTreeMaxItemsInAutoPrefix;
-      this.analyzerPositionGap = other.analyzerPositionGap;
-      this.analyzerOffsetGap = other.analyzerOffsetGap;
-      this.minTokenLength = other.minTokenLength;
-      this.maxTokenLength = other.maxTokenLength;
-      this.maxTokenCount = other.maxTokenCount;
-      this.stored = other.stored;
-      this.sortable = other.sortable;
-      this.sortReversed = other.sortReversed;
-      this.sortMissingLast = other.sortMissingLast;
-      this.fastRanges = other.fastRanges;
-      this.multiValued = other.multiValued;
-      this.indexNorms = other.indexNorms;
-      this.storeTermVectors = other.storeTermVectors;
-      this.storeTermVectorPositions = other.storeTermVectorPositions;
-      this.storeTermVectorOffsets = other.storeTermVectorOffsets;
-      this.storeTermVectorPayloads = other.storeTermVectorPayloads;
-      this.indexOptions = other.indexOptions;
-      this.indexOptionsSet = other.indexOptionsSet;
-      this.docValuesFormat = other.docValuesFormat;
-      this.postingsFormat = other.postingsFormat;
-      this.highlighted = other.highlighted;
-      this.queryAnalyzer = other.queryAnalyzer;
-      this.indexAnalyzer = other.indexAnalyzer;
-      this.similarity = other.similarity;
-      this.wrappedIndexAnalyzer = other.wrappedIndexAnalyzer;
-      this.wrappedQueryAnalyzer = other.wrappedQueryAnalyzer;
-    }
-
     volatile ValueType valueType = ValueType.NONE;
     volatile DocValuesType docValuesType = DocValuesType.NONE;
     private volatile boolean docValuesTypeSet;
@@ -342,6 +243,12 @@ public class FieldTypes {
     // True if the term is unique across all documents (e.g. a primary key field):
     volatile Boolean isUnique;
 
+    // True when Document.addStoredXXX was used:
+    volatile Boolean storedOnly;
+
+    // Only used for ATOM:
+    volatile Boolean isBinary;
+
     // Expert: settings we pass to BlockTree to control how many terms are allowed in each block and auto-prefix term
     volatile Integer blockTreeMinItemsInBlock;
     volatile Integer blockTreeMaxItemsInBlock;
@@ -352,8 +259,6 @@ public class FieldTypes {
     volatile Integer analyzerPositionGap;
     volatile Integer analyzerOffsetGap;
 
-    // nocommit should we default max token length to ... 256?
-
     // Min/max token length, or null if there are no limits:
     volatile Integer minTokenLength;
     volatile Integer maxTokenLength;
@@ -369,6 +274,8 @@ public class FieldTypes {
     private volatile Boolean sortable;
     private volatile Boolean sortReversed;
     private volatile Boolean sortMissingLast = Boolean.TRUE;
+    private volatile SortedNumericSelector.Type numericSelector = SortedNumericSelector.Type.MIN;
+    private volatile SortedSetSelector.Type sortedSetSelector = SortedSetSelector.Type.MIN;
 
     // Whether this field's values should be indexed for fast ranges (using numeric field for now):
     private volatile Boolean fastRanges;
@@ -379,6 +286,9 @@ public class FieldTypes {
     // Whether this field's norms are indexed:
     private volatile Boolean indexNorms;
 
+    // Bit width for a big int field:
+    volatile Integer bigIntByteWidth;
+
     private volatile Boolean storeTermVectors;
     private volatile Boolean storeTermVectorPositions;
     private volatile Boolean storeTermVectorOffsets;
@@ -388,8 +298,7 @@ public class FieldTypes {
     private volatile IndexOptions indexOptions = IndexOptions.NONE;
     private volatile boolean indexOptionsSet;
 
-    // nocommit: not great that we can't also set other formats... but we need per-field wrappers to do this, or we need to move
-    // "per-field-ness" of these formats into here, or something:
+    // TODO: not great that we can't also set other formats:
     private volatile String postingsFormat;
     private volatile String docValuesFormat;
 
@@ -404,10 +313,307 @@ public class FieldTypes {
     private volatile Analyzer wrappedIndexAnalyzer;
     private volatile Analyzer wrappedQueryAnalyzer;
 
+    volatile Boolean reversedTerms;
+
     Locale sortLocale;
     Collator sortCollator;
     SortKey sortKey;
 
+    public FieldType(String name) {
+      this(name, Version.LATEST);
+    }
+
+    public FieldType(String name, Version version) {
+      this.name = name;
+      this.createdVersion = version;
+    }
+
+    /** Copy constructor. */
+    FieldType(FieldType other) {
+      this.name = other.name;
+      this.createdVersion = other.createdVersion;
+      this.valueType = other.valueType;
+      this.docValuesType = other.docValuesType;
+      this.docValuesTypeSet = other.docValuesTypeSet;
+      this.isUnique = other.isUnique;
+      this.storedOnly = other.storedOnly;
+      this.isBinary = other.isBinary;
+      this.blockTreeMinItemsInBlock = other.blockTreeMinItemsInBlock;
+      this.blockTreeMaxItemsInBlock = other.blockTreeMaxItemsInBlock;
+      this.blockTreeMinItemsInAutoPrefix = other.blockTreeMinItemsInAutoPrefix;
+      this.blockTreeMaxItemsInAutoPrefix = other.blockTreeMaxItemsInAutoPrefix;
+      this.analyzerPositionGap = other.analyzerPositionGap;
+      this.analyzerOffsetGap = other.analyzerOffsetGap;
+      this.minTokenLength = other.minTokenLength;
+      this.maxTokenLength = other.maxTokenLength;
+      this.maxTokenCount = other.maxTokenCount;
+      this.consumeAllTokens = other.consumeAllTokens;
+      this.stored = other.stored;
+      this.sortable = other.sortable;
+      this.sortReversed = other.sortReversed;
+      this.sortMissingLast = other.sortMissingLast;
+      this.numericSelector = other.numericSelector;
+      this.sortedSetSelector = other.sortedSetSelector;
+      this.fastRanges = other.fastRanges;
+      this.multiValued = other.multiValued;
+      this.indexNorms = other.indexNorms;
+      this.bigIntByteWidth = other.bigIntByteWidth;
+      this.storeTermVectors = other.storeTermVectors;
+      this.storeTermVectorPositions = other.storeTermVectorPositions;
+      this.storeTermVectorOffsets = other.storeTermVectorOffsets;
+      this.storeTermVectorPayloads = other.storeTermVectorPayloads;
+      this.indexOptions = other.indexOptions;
+      this.indexOptionsSet = other.indexOptionsSet;
+      this.postingsFormat = other.postingsFormat;
+      this.docValuesFormat = other.docValuesFormat;
+      this.highlighted = other.highlighted;
+      this.queryAnalyzer = other.queryAnalyzer;
+      this.indexAnalyzer = other.indexAnalyzer;
+      this.similarity = other.similarity;
+      this.wrappedIndexAnalyzer = other.wrappedIndexAnalyzer;
+      this.wrappedQueryAnalyzer = other.wrappedQueryAnalyzer;
+      this.reversedTerms = other.reversedTerms;
+      this.sortLocale = other.sortLocale;
+      this.sortCollator = other.sortCollator;
+      this.sortKey = other.sortKey;
+    }
+
+    public synchronized void merge(FieldType other) {
+      assert name.equals(other.name);
+
+      // nocommit more need test coverage here
+      if (other.valueType != ValueType.NONE) {
+        if (valueType == ValueType.NONE) {
+          valueType = other.valueType;
+        } else if (other.valueType != valueType) {
+          illegalState(name, "cannot change value type from " + valueType + " to " + other.valueType);
+        }
+      }
+      if (other.docValuesTypeSet) {
+        if (docValuesTypeSet == false) {
+          docValuesType = other.docValuesType;  
+          docValuesTypeSet = true;
+        } else if (other.docValuesType != docValuesType) {
+          illegalState(name, "cannot change docValuesType from " + docValuesType + " to " + other.docValuesType);
+        }
+      }
+      if (other.isUnique != null) {
+        if (isUnique == null) {
+          isUnique = other.isUnique;
+        } else if (other.isUnique != isUnique) {
+          illegalState(name, "cannot change isUnique from " + isUnique + " to " + other.isUnique);
+        }
+      }
+      if (other.storedOnly != null) {
+        if (storedOnly == null) {
+          storedOnly = other.storedOnly;
+        } else if (other.storedOnly != storedOnly) {
+          illegalState(name, "cannot change storedOnly from " + storedOnly + " to " + other.storedOnly);
+        }
+      }
+      if (other.isBinary != null) {
+        if (isBinary == null) {
+          isBinary = other.isBinary;
+        } else if (other.isBinary != isBinary) {
+          illegalState(name, "cannot change isBinary from " + isBinary + " to " + other.isBinary);
+        }
+      }
+      if (other.blockTreeMinItemsInBlock != null) {
+        blockTreeMinItemsInBlock = other.blockTreeMinItemsInBlock;
+      }
+      if (other.blockTreeMaxItemsInBlock != null) {
+        blockTreeMaxItemsInBlock = other.blockTreeMaxItemsInBlock;
+      }
+      if (other.blockTreeMinItemsInAutoPrefix != null) {
+        blockTreeMinItemsInAutoPrefix = other.blockTreeMinItemsInAutoPrefix;
+      }
+      if (other.blockTreeMaxItemsInAutoPrefix != null) {
+        blockTreeMaxItemsInAutoPrefix = other.blockTreeMaxItemsInAutoPrefix;
+      }
+
+      if (other.analyzerPositionGap != null) {
+        if (analyzerPositionGap == null) {
+          analyzerPositionGap = other.analyzerPositionGap;
+        } else if (other.analyzerPositionGap.equals(analyzerPositionGap) == false) {
+          illegalState(name, "cannot change analyzerPositionGap from " + analyzerPositionGap + " to " + other.analyzerPositionGap);
+        }
+      }
+      if (other.analyzerOffsetGap != null) {
+        if (analyzerOffsetGap == null) {
+          analyzerOffsetGap = other.analyzerOffsetGap;
+        } else if (other.analyzerOffsetGap.equals(analyzerOffsetGap) == false) {
+          illegalState(name, "cannot change analyzerOffsetGap from " + analyzerOffsetGap + " to " + other.analyzerOffsetGap);
+        }
+      }
+      if (other.minTokenLength != null) {
+        minTokenLength = other.minTokenLength;
+      }
+      if (other.maxTokenLength != null) {
+        maxTokenLength = other.maxTokenLength;
+      }
+      if (other.maxTokenCount != null) {
+        maxTokenCount = other.maxTokenCount;
+      }
+      if (other.consumeAllTokens != null) {
+        consumeAllTokens = other.consumeAllTokens;
+      }
+      if (other.stored != null) {
+        stored = other.stored;
+      }
+      if (other.sortable != null) {
+        if (sortable == null) {
+          sortable = other.sortable;
+        } else if (other.sortable == Boolean.FALSE) {
+          sortable = other.sortable;
+        } else if (sortable == Boolean.FALSE) {
+          illegalState(name, "sorting was already disabled");
+        }
+        if (other.sortReversed != null) {
+          sortReversed = other.sortReversed;
+        }
+        if (other.sortMissingLast != null) {
+          sortMissingLast = other.sortMissingLast;
+        }
+      }
+
+      if (other.numericSelector != null) {
+        numericSelector = other.numericSelector;
+      }
+      if (other.sortedSetSelector != null) {
+        sortedSetSelector = other.sortedSetSelector;
+      }
+      if (other.fastRanges != null) {
+        if (fastRanges == null) {
+          fastRanges = other.fastRanges;
+        } else if (other.fastRanges == Boolean.FALSE) {
+          fastRanges = Boolean.FALSE;
+        } else if (fastRanges == Boolean.FALSE) {
+          illegalState(name, "fastRanges was already disabled");
+        }
+      }
+      if (other.multiValued != null) {
+        if (multiValued == null) {
+          multiValued = other.multiValued;
+        } else if (other.multiValued != multiValued) {
+          illegalState(name, "cannot change multiValued from " + multiValued + " to " + other.multiValued);
+        }
+      }
+
+      if (other.indexNorms != null) {
+        if (indexNorms == null) {
+          indexNorms = other.indexNorms;
+        } else if (other.indexNorms == Boolean.FALSE) {
+          indexNorms = Boolean.FALSE;
+        } else if (indexNorms == Boolean.FALSE) {
+          illegalState(name, "norms were already disabled");
+        }
+      }
+
+      if (other.bigIntByteWidth != null) {
+        if (bigIntByteWidth == null) {
+          bigIntByteWidth = other.bigIntByteWidth;
+        } else if (bigIntByteWidth.equals(other.bigIntByteWidth) == false) {
+          illegalState(name, "cannot change bigIntByteWidth from " + bigIntByteWidth + " to " + other.bigIntByteWidth);
+        }
+      }
+
+      if (other.storeTermVectors != null) {
+        storeTermVectors = other.storeTermVectors;
+
+        if (other.storeTermVectorPositions != null) {
+          storeTermVectorPositions = other.storeTermVectorPositions;
+        }
+        if (other.storeTermVectorOffsets != null) {
+          storeTermVectorOffsets = other.storeTermVectorOffsets;
+        }
+        if (other.storeTermVectorPayloads != null) {
+          storeTermVectorPayloads = other.storeTermVectorPayloads;
+        }
+      }
+
+      if (other.indexOptionsSet) {
+        if (indexOptionsSet == false) {
+          indexOptions = other.indexOptions;
+          indexOptionsSet = true;
+        } else if (indexOptions.compareTo(other.indexOptions) >= 0) {
+          indexOptions = other.indexOptions;
+        } else {
+          illegalState(name, "cannot upgrade indexOptions from " + indexOptions + " to " + other.indexOptions);
+        }
+      }
+
+      if (other.postingsFormat != null) {
+        postingsFormat = other.postingsFormat;
+      }
+
+      if (other.docValuesFormat != null) {
+        docValuesFormat = other.docValuesFormat;
+      }
+
+      if (other.highlighted != null) {
+        if (highlighted == null) {
+          highlighted = other.highlighted;
+        } else if (other.highlighted == Boolean.FALSE) {
+          highlighted = Boolean.FALSE;
+        } else if (highlighted == Boolean.FALSE) {
+          illegalState(name, "highlighting was already disabled");
+        }
+      }
+
+      if (other.queryAnalyzer != null) {
+        if (queryAnalyzer == null) {
+          queryAnalyzer = other.queryAnalyzer;
+        } else if (queryAnalyzer != other.queryAnalyzer) {
+          illegalState(name, "queryAnalyzer was already set");
+        }
+      }
+
+      if (other.indexAnalyzer != null) {
+        if (indexAnalyzer == null) {
+          indexAnalyzer = other.indexAnalyzer;
+        } else if (indexAnalyzer != other.indexAnalyzer) {
+          illegalState(name, "indexAnalyzer was already set");
+        }
+      }
+
+      if (other.similarity != null) {
+        similarity = other.similarity;
+      }
+
+      if (other.wrappedIndexAnalyzer != null) {
+        wrappedIndexAnalyzer = other.wrappedIndexAnalyzer;
+      }
+      if (other.wrappedQueryAnalyzer != null) {
+        wrappedQueryAnalyzer = other.wrappedQueryAnalyzer;
+      }
+      if (other.reversedTerms != null) {
+        if (reversedTerms == null) {
+          reversedTerms = other.reversedTerms;
+        } else if (other.reversedTerms != reversedTerms) {
+          illegalState(name, "can only setReversedTerms before the field is indexed");
+        }
+      }
+      if (other.sortLocale != null) {
+        if (sortLocale == null) {
+          sortLocale = other.sortLocale;
+          sortCollator = other.sortCollator;
+        } else if (sortLocale.equals(other.sortLocale) == false) {
+          if (valueType == null) {
+            sortLocale = other.sortLocale;
+            sortCollator = other.sortCollator;
+          } else {
+            illegalState(name, "sortLocale can only be set before indexing");
+          }
+        }
+      }
+
+      if (other.sortKey != null) {
+        sortKey = other.sortKey;
+      }
+
+      changed(false);
+    }
+
     boolean validate() {
       switch (valueType) {
       case NONE:
@@ -484,7 +690,6 @@ public class FieldTypes {
           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; got " + indexOptions);
         }
         if (minTokenLength != null) {
@@ -495,9 +700,6 @@ 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");
         }
@@ -507,9 +709,6 @@ public class FieldTypes {
         if (indexNorms == Boolean.TRUE) {
           illegalState(name, "type " + valueType + " cannot index norms");
         }
-        if (indexOptions != IndexOptions.NONE && indexOptions.compareTo(IndexOptions.DOCS) > 0 && multiValued != Boolean.TRUE) {
-          illegalState(name, "type " + valueType + " can only be indexed as DOCS; got " + indexOptions);
-        }
         if (maxTokenCount != null) {
           illegalState(name, "type " + valueType + " cannot set max token count");
         }
@@ -535,8 +734,6 @@ public class FieldTypes {
         throw new AssertionError("missing value type in switch");
       }
 
-      // nocommit more checks
-
       if (sortKey != null && valueType != ValueType.ATOM) {
         illegalState(name, "sortKey can only be set for ATOM fields; got value type=" + valueType);
       }
@@ -605,6 +802,15 @@ public class FieldTypes {
         }
       }
 
+      if (reversedTerms == Boolean.TRUE) {
+        if (indexOptions == IndexOptions.NONE) {
+          illegalState(name, "can only reverse terms if the field is indexed");
+        }
+        if (valueType != ValueType.SHORT_TEXT && valueType != ValueType.TEXT && valueType != ValueType.ATOM) {
+          illegalState(name, "can only reverse terms for text and short_text value type; got value type=" + valueType);
+        }
+      }
+
       // nocommit must check that if fastRanges is on, you have a PF that supports it
       
       if (fastRanges == Boolean.TRUE && indexOptions != IndexOptions.DOCS) {
@@ -638,7 +844,7 @@ public class FieldTypes {
       }
 
       if (highlighted == Boolean.TRUE) {
-        if (valueType != ValueType.TEXT && valueType != ValueType.SHORT_TEXT) {
+        if (valueType != ValueType.TEXT && valueType != ValueType.SHORT_TEXT && valueType != ValueType.ATOM) {
           illegalState(name, "can only enable highlighting for TEXT or SHORT_TEXT fields; got value type=" + valueType);
         }
         if (indexOptions != IndexOptions.NONE && indexOptions != IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) {
@@ -650,11 +856,10 @@ public class FieldTypes {
     }
 
     private boolean needsWrapping() {
-      return minTokenLength != null || maxTokenCount != null;
+      return minTokenLength != null || maxTokenCount != null || reversedTerms == Boolean.TRUE;
     }
 
     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);
@@ -671,15 +876,356 @@ public class FieldTypes {
           wrappedQueryAnalyzer = null;
         }
       } else {
-        wrappedIndexAnalyzer = null;
-        wrappedQueryAnalyzer = null;
+        wrappedIndexAnalyzer = indexAnalyzer;
+        wrappedQueryAnalyzer = queryAnalyzer;
       }
     }
 
-    private Analyzer wrapAnalyzer(final Analyzer in) {
-      return new AnalyzerWrapper(in.getReuseStrategy()) {
-        @Override
-        protected Analyzer getWrappedAnalyzer(String fieldName) {
+    void setDefaults() {
+      switch (valueType) {
+      case NONE:
+        // bug
+        throw new AssertionError("valueType should not be NONE");
+      case INT:
+      case HALF_FLOAT:
+      case FLOAT:
+      case LONG:
+      case DOUBLE:
+      case BIG_INT:
+      case DATE:
+        if (highlighted == null) {
+          highlighted = Boolean.FALSE;
+        }
+        if (storeTermVectors == null) {
+          storeTermVectors = Boolean.FALSE;
+        }
+        if (sortable == null) {
+          if (valueType == ValueType.BIG_INT) {
+            if (docValuesTypeSet == false || docValuesType == DocValuesType.SORTED || docValuesType == DocValuesType.SORTED_SET) {
+              sortable = Boolean.TRUE;
+            } else {
+              sortable = Boolean.FALSE;
+            }
+          } else {
+            if (docValuesTypeSet == false || docValuesType == DocValuesType.NUMERIC || docValuesType == DocValuesType.SORTED_NUMERIC) {
+              sortable = Boolean.TRUE;
+            } else {
+              sortable = Boolean.FALSE;
+            }
+          }
+        }
+        if (multiValued == null) {
+          multiValued = Boolean.FALSE;
+        }
+        if (stored == null) {
+          stored = Boolean.TRUE;
+        }
+        if (indexOptionsSet == false) {
+          indexOptions = IndexOptions.DOCS;
+          indexOptionsSet = true;
+        }
+        if (docValuesTypeSet == false) {
+          if (sortable == Boolean.TRUE) {
+            if (valueType == ValueType.BIG_INT) {
+              if (multiValued == Boolean.TRUE) {
+                docValuesType = DocValuesType.SORTED_SET;
+              } else {
+                docValuesType = DocValuesType.SORTED;
+              }
+            } else {
+              if (multiValued == Boolean.TRUE) {
+                docValuesType = DocValuesType.SORTED_NUMERIC;
+              } else {
+                docValuesType = DocValuesType.NUMERIC;
+              }
+            }
+          }
+          docValuesTypeSet = true;
+        }
+        if (fastRanges == null) {
+          if (indexOptions != IndexOptions.NONE) {
+            fastRanges = Boolean.TRUE;
+          } else {
+            fastRanges = Boolean.FALSE;
+          }
+        }
+        if (indexNorms == null) {
+          indexNorms = Boolean.FALSE;
+        }
+        if (isUnique == null) {
+          isUnique = Boolean.FALSE;
+        }
+        break;
+
+      case SHORT_TEXT:
+        if (highlighted == null) {
+          highlighted = Boolean.TRUE;
+        }
+        if (storeTermVectors == null) {
+          storeTermVectors = Boolean.FALSE;
+        }
+        if (sortable == null) {
+          if (docValuesTypeSet == false || docValuesType == DocValuesType.SORTED || docValuesType == DocValuesType.SORTED_SET) {
+            sortable = Boolean.TRUE;
+          } else {
+            sortable = Boolean.FALSE;
+          }
+        }
+        if (multiValued == null) {
+          multiValued = Boolean.FALSE;
+        }
+        if (stored == null) {
+          stored = Boolean.TRUE;
+        }
+        if (docValuesTypeSet == false) {
+          if (sortable == Boolean.TRUE) {
+            if (multiValued == Boolean.TRUE) {
+              docValuesType = DocValuesType.SORTED_SET;
+            } else {
+              docValuesType = DocValuesType.SORTED;
+            }
+          }
+          docValuesTypeSet = true;
+        }
+        if (indexOptionsSet == false) {
+          if (highlighted) {
+            indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS;
+          } else {
+            indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
+          }
+          indexOptionsSet = true;
+        }
+        if (fastRanges == null) {
+          fastRanges = Boolean.FALSE;
+        }
+        if (indexNorms == null) {
+          indexNorms = Boolean.FALSE;
+        }
+        if (isUnique == null) {
+          isUnique = Boolean.FALSE;
+        }
+        break;
+
+      case ATOM:
+      case INET_ADDRESS:
+        if (highlighted == null) {
+          highlighted = Boolean.FALSE;
+        }
+        if (storeTermVectors == null) {
+          storeTermVectors = Boolean.FALSE;
+        }
+        if (sortable == null) {
+          if (docValuesTypeSet == false || docValuesType == DocValuesType.SORTED || docValuesType == DocValuesType.SORTED_SET) {
+            sortable = Boolean.TRUE;
+          } else {
+            sortable = Boolean.FALSE;
+          }
+        }
+        if (multiValued == null) {
+          multiValued = Boolean.FALSE;
+        }
+        if (stored == null) {
+          stored = Boolean.TRUE;
+        }
+        if (indexOptionsSet == false) { 
+          if (highlighted) {
+            indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS;
+          } else {
+            indexOptions = IndexOptions.DOCS;
+          }
+          indexOptionsSet = true;
+        }
+        if (docValuesTypeSet == false) {
+          if (sortable == Boolean.TRUE) {
+            if (multiValued == Boolean.TRUE) {
+              docValuesType = DocValuesType.SORTED_SET;
+            } else {
+              docValuesType = DocValuesType.SORTED;
+            }
+          }
+          docValuesTypeSet = true;
+        }
+        if (fastRanges == null) {
+          if (indexOptions != IndexOptions.NONE) {
+            fastRanges = Boolean.TRUE;
+          } else {
+            fastRanges = Boolean.FALSE;
+          }
+        }
+        if (indexNorms == null) {
+          indexNorms = Boolean.FALSE;
+        }
+        if (isUnique == null) {
+          isUnique = Boolean.FALSE;
+        }
+        break;
+
+      case BINARY:
+        if (highlighted == null) {
+          highlighted = Boolean.FALSE;
+        }
+        if (storeTermVectors == null) {
+          storeTermVectors = Boolean.FALSE;
+        }
+        if (sortable == null) {
+          if (docValuesTypeSet == false || docValuesType == DocValuesType.SORTED || docValuesType == DocValuesType.SORTED_SET) {
+            sortable = Boolean.TRUE;
+          } else {
+            sortable = Boolean.FALSE;
+          }
+        }
+        if (multiValued == null) {
+          multiValued = Boolean.FALSE;
+        }
+        if (stored == null) {
+          stored = Boolean.TRUE;
+        }
+        if (indexOptionsSet == false) {
+          assert indexOptions == IndexOptions.NONE;
+          indexOptionsSet = true;
+        }
+        if (docValuesTypeSet == false) {
+          if (sortable == Boolean.TRUE) {
+            if (multiValued == Boolean.TRUE) {
+              docValuesType = DocValuesType.SORTED_SET;
+            } else {
+              docValuesType = DocValuesType.SORTED;
+            }
+          } else {
+            docValuesType = DocValuesType.BINARY;
+          }
+          docValuesTypeSet = true;
+        }
+        if (fastRanges == null) {
+          fastRanges = Boolean.FALSE;
+        }
+        if (indexNorms == null) {
+          indexNorms = Boolean.FALSE;
+        }
+        if (isUnique == null) {
+          isUnique = Boolean.FALSE;
+        }
+        break;
+
+      case TEXT:
+        if (highlighted == null) {
+          highlighted = Boolean.TRUE;
+        }
+        if (storeTermVectors == null) {
+          storeTermVectors = Boolean.FALSE;
+        }
+        if (sortable == null) {
+          sortable = Boolean.FALSE;
+        }
+        if (multiValued == null) {
+          multiValued = Boolean.FALSE;
+        }
+        if (stored == null) {
+          stored = Boolean.TRUE;
+        }
+        if (indexOptionsSet == false) {
+          if (highlighted) {
+            indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS;
+          } else {
+            indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
+          }
+          indexOptionsSet = true;
+        }
+
+        assert docValuesType == DocValuesType.NONE;
+        docValuesTypeSet = true;
+
+        if (fastRanges == null) {
+          fastRanges = Boolean.FALSE;
+        }
+        if (indexNorms == null) {
+          indexNorms = Boolean.TRUE;
+        }
+        if (isUnique == null) {
+          isUnique = Boolean.FALSE;
+        }
+        break;
+    
+      case BOOLEAN:
+        if (highlighted == null) {
+          highlighted = Boolean.FALSE;
+        }
+        if (storeTermVectors == null) {
+          storeTermVectors = Boolean.FALSE;
+        }
+        if (sortable == null) {
+          sortable = Boolean.TRUE;
+        }
+        if (multiValued == null) {
+          multiValued = Boolean.FALSE;
+        }
+        if (stored == null) {
+          stored = Boolean.TRUE;
+        }
+        if (indexOptionsSet == false) {
+          // validate enforces this:
+          assert highlighted == false;
+          indexOptions = IndexOptions.DOCS;
+          indexOptionsSet = true;
+        }
+        if (docValuesTypeSet == false) {
+          if (sortable == Boolean.TRUE) {
+            if (multiValued == Boolean.TRUE) {
+              docValuesType = DocValuesType.SORTED_NUMERIC;
+            } else {
+              docValuesType = DocValuesType.NUMERIC;
+            }
+          }
+          docValuesTypeSet = true;
+        }
+        if (fastRanges == null) {
+          fastRanges = Boolean.FALSE;
+        }
+        if (indexNorms == null) {
+          indexNorms = Boolean.FALSE;
+        }
+        if (isUnique == null) {
+          isUnique = Boolean.FALSE;
+        }
+        break;
+
+      default:
+        throw new AssertionError("missing value type in switch");
+      }
+
+      if (fastRanges == Boolean.TRUE) {
+        if (blockTreeMinItemsInAutoPrefix == null) {
+          blockTreeMinItemsInAutoPrefix = BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE;
+          blockTreeMaxItemsInAutoPrefix = BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE;
+        }
+      }
+
+      assert name != null;
+      assert createdVersion != null;
+      assert valueType != null;
+      assert docValuesTypeSet;
+      assert docValuesType != null;
+      assert isUnique != null;
+      assert storedOnly != null;
+      assert valueType != ValueType.ATOM || isBinary != null;
+      assert indexOptionsSet;
+      assert indexOptions != null;
+      assert stored != null;
+      assert sortable != null;
+      assert fastRanges != null;
+      assert multiValued != null;
+      assert indexOptions == IndexOptions.NONE || indexNorms != null;
+      assert highlighted != null;
+      assert storeTermVectors != null;
+
+      // setDefaults() should never create an invalid state:
+      validate();
+    } 
+
+    private Analyzer wrapAnalyzer(final Analyzer in) {
+      return new AnalyzerWrapper(in.getReuseStrategy()) {
+        @Override
+        protected Analyzer getWrappedAnalyzer(String fieldName) {
           return in;
         }
 
@@ -691,10 +1237,21 @@ public class FieldTypes {
                                    minTokenLength.intValue(),
                                    maxTokenLength.intValue());
           }
+
           if (maxTokenCount != null) {
             end = new LimitTokenCountFilter(end, maxTokenCount.intValue(), consumeAllTokens.booleanValue());
           }
 
+          if (reversedTerms == Boolean.TRUE) {
+            try {
+              Class c = Class.forName("org.apache.lucene.analysis.reverse.ReverseStringFilter");
+              Constructor init = c.getConstructor(new Class[] {TokenStream.class});
+              end = (TokenStream) init.newInstance(end);
+            } catch (ReflectiveOperationException roe) {
+              throw new IllegalStateException("could not locate ReverseStringFilter; ensure Lucene's analysis module is on your CLASSPATH", roe);
+            }
+          }
+
           return new TokenStreamComponents(components.getTokenizer(), end);
         }
       };
@@ -708,6 +1265,9 @@ public class FieldTypes {
       b.append("\":\n");
       b.append("  value type: ");
       b.append(valueType);
+      if (valueType == ValueType.ATOM && isBinary == Boolean.TRUE) {
+        b.append(" (binary)");
+      }
       b.append('\n');
 
       if (blockTreeMinItemsInBlock != null) {
@@ -756,6 +1316,16 @@ public class FieldTypes {
             b.append(" (missing: first)");
           }
         }
+
+        if (multiValued == Boolean.TRUE) {
+          if (isNumericType(valueType)) {
+            if (numericSelector != null) {
+              b.append(" (numericSelector: " + numericSelector + ")");
+            }
+          } else if (sortedSetSelector != null) {
+            b.append(" (sortedSetSelector: " + sortedSetSelector + ")");
+          }
+        }
       } else {
         b.append("unset");
       }
@@ -765,13 +1335,17 @@ public class FieldTypes {
       if (fastRanges != null) {
         b.append(fastRanges);
         if (fastRanges == Boolean.TRUE) {
+          b.append(" (auto-prefix blocks: ");
           if (blockTreeMinItemsInAutoPrefix != null) {
-            b.append(" (auto-prefix blocks: ");
             b.append(blockTreeMinItemsInAutoPrefix);
             b.append(" - ");
             b.append(blockTreeMaxItemsInAutoPrefix);
-            b.append(")");
+          } else {
+            b.append(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE);
+            b.append(" - ");
+            b.append(BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
           }
+          b.append(")");
         }
       } else {
         b.append("unset");
@@ -881,7 +1455,6 @@ public class FieldTypes {
     }
 
     void write(DataOutput out) throws IOException {
-      // nocommit under codec control instead?
       out.writeString(name);
 
       out.writeVInt(createdVersion.major);
@@ -977,17 +1550,24 @@ public class FieldTypes {
       writeNullableBoolean(out, sortable);
       writeNullableBoolean(out, sortReversed);
       writeNullableBoolean(out, sortMissingLast);
+      out.writeString(numericSelector.toString());
+      out.writeString(sortedSetSelector.toString());
       writeNullableBoolean(out, multiValued);
       writeNullableBoolean(out, indexNorms);
+      writeNullableBoolean(out, reversedTerms);
+      writeNullableInteger(out, bigIntByteWidth);
       writeNullableBoolean(out, fastRanges);
       writeNullableBoolean(out, storeTermVectors);
       writeNullableBoolean(out, storeTermVectorPositions);
       writeNullableBoolean(out, storeTermVectorOffsets);
       writeNullableBoolean(out, storeTermVectorPayloads);
       writeNullableBoolean(out, isUnique);
+      writeNullableBoolean(out, storedOnly);
+      writeNullableBoolean(out, isBinary);
 
       if (sortLocale != null) {
         out.writeByte((byte) 1);
+        // nocommit this is not sufficient right?  need to use the builder?
         writeNullableString(out, sortLocale.getLanguage());
         writeNullableString(out, sortLocale.getCountry());
         writeNullableString(out, sortLocale.getVariant());
@@ -1026,7 +1606,6 @@ 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());
 
@@ -1126,14 +1705,20 @@ public class FieldTypes {
       sortable = readNullableBoolean(in);
       sortReversed = readNullableBoolean(in);
       sortMissingLast = readNullableBoolean(in);
+      numericSelector = SortedNumericSelector.Type.valueOf(in.readString());
+      sortedSetSelector = SortedSetSelector.Type.valueOf(in.readString());
       multiValued = readNullableBoolean(in);
       indexNorms = readNullableBoolean(in);
+      reversedTerms = readNullableBoolean(in);
+      bigIntByteWidth = readNullableInteger(in);
       fastRanges = readNullableBoolean(in);
       storeTermVectors = readNullableBoolean(in);
       storeTermVectorPositions = readNullableBoolean(in);
       storeTermVectorOffsets = readNullableBoolean(in);
       storeTermVectorPayloads = readNullableBoolean(in);
       isUnique = readNullableBoolean(in);
+      storedOnly = readNullableBoolean(in);
+      isBinary = readNullableBoolean(in);
       b = in.readByte();
       if (b == 1) {
         String language = readNullableString(in);
@@ -1207,15 +1792,10 @@ public class FieldTypes {
     }
   }
 
-  // nocommit need test that you cannot .addStored after already .addLargeText(TokenStream)?
-
-  // nocommit move to oal.index and remove these ctors, so you must ask IW or IR for the FieldTypes
-
   /** Only invoked by IndexWriter directly.
    *
    * @lucene.internal */
   // nocommit lock this down so only IW can create?
-  // nocommit throw corrupt ie if not isNewIndex but schema is missing
   public FieldTypes(IndexWriter writer, boolean isNewIndex, Analyzer defaultIndexAnalyzer, Similarity defaultSimilarity) throws IOException {
     this.readOnly = false;
     indexCreatedVersion = loadFields(writer.getCommitData(), isNewIndex);
@@ -1234,26 +1814,53 @@ public class FieldTypes {
 
   public FieldTypes(FieldTypes other) {
     readOnly = true;
-    indexCreatedVersion = other.indexCreatedVersion;
     this.defaultIndexAnalyzer = null;
-    this.defaultQueryAnalyzer = other.defaultQueryAnalyzer;
-    this.defaultSimilarity = other.defaultSimilarity;
-    addAll(other);
+    if (other != null) {
+      indexCreatedVersion = other.indexCreatedVersion;
+      this.defaultQueryAnalyzer = other.defaultQueryAnalyzer;
+      this.defaultSimilarity = other.defaultSimilarity;
+      addAll(other);
+    } else {
+      indexCreatedVersion = Version.LATEST;
+      this.defaultQueryAnalyzer = null;
+      this.defaultSimilarity = null;
+    }
+  }
+
+  public FieldTypes(FieldTypes other, Iterable<String> fieldsToKeep) {
+    readOnly = true;
+    this.defaultIndexAnalyzer = null;
+    if (other != null) {
+      this.defaultQueryAnalyzer = other.defaultQueryAnalyzer;
+      this.defaultSimilarity = other.defaultSimilarity;
+      indexCreatedVersion = other.indexCreatedVersion;
+      for(String field : fieldsToKeep) {
+        FieldType fieldType = other.fields.get(field);
+        if (fieldType != null) {
+          fields.put(field, new FieldType(fieldType));
+        }
+      }
+      FieldType fieldType = other.fields.get(FIELD_NAMES_FIELD);
+      if (fieldType != null) {
+        fields.put(FIELD_NAMES_FIELD, new FieldType(fieldType));
+      }
+      
+    } else {
+      addFieldNamesField();
+      this.defaultQueryAnalyzer = null;
+      this.defaultSimilarity = null;
+      indexCreatedVersion = Version.LATEST;
+    }
   }
 
-  // nocommit messy we steal this from commitdata namespace...
-  /** Key used to store the field types inside {@link IndexWriter#setCommitData}. */
-  public static final String FIELD_TYPES_KEY = "FieldTypes";
-  
   private synchronized Version loadFields(Map<String,String> commitUserData, boolean isNewIndex) throws IOException {
-    // nocommit must deserialize current fields from commit data
     String currentFieldTypes = commitUserData.get(FIELD_TYPES_KEY);
     if (currentFieldTypes != null) {
       return readFromString(currentFieldTypes);
     } else if (isNewIndex == false) {
+      // Index already exists, but no FieldTypes
       // nocommit must handle back compat here
       // throw new CorruptIndexException("FieldTypes is missing from this index", "CommitUserData");
-      System.out.println("  not new, null");
       enableExistsFilters = false;
       return Version.LATEST;
     } else {
@@ -1267,11 +1874,14 @@ public class FieldTypes {
 
     FieldType fieldType = new FieldType(FIELD_NAMES_FIELD);
     fields.put(FIELD_NAMES_FIELD, fieldType);
-    fieldType.multiValued = Boolean.TRUE;
     fieldType.valueType = ValueType.ATOM;
+    fieldType.multiValued = Boolean.TRUE;
     fieldType.sortable = Boolean.FALSE;
     fieldType.stored = Boolean.FALSE;
-    setDefaults(fieldType);
+    fieldType.storedOnly = Boolean.FALSE;
+    fieldType.fastRanges = Boolean.FALSE;
+    fieldType.isBinary = Boolean.FALSE;
+    fieldType.setDefaults();
   }
 
   private synchronized FieldType newFieldType(String fieldName) {
@@ -1303,8 +1913,7 @@ public class FieldTypes {
     return bytesOut;
   }
 
-  // nocommit messy: this is because we stuff the schema into user data; we should instead safe "directly" somewhere (new gen'd file?
-  // inside segments_N?)
+  // nocommit: let codec control this format?  should we write directly into segments_N?  or into a new separately gen'd file?
 
   /** Encodes byte[] to 7-bit clean chars (ascii). */
   private static String bytesToString(byte[] bytesIn) {
@@ -1332,6 +1941,7 @@ public class FieldTypes {
   }
 
   public synchronized void setPostingsFormat(String fieldName, String postingsFormat) {
+    // nocommit can we prevent this, if our codec isn't used by IW?
     try {
       // Will throw exception if this postingsFormat is unrecognized:
       PostingsFormat.forName(postingsFormat);
@@ -1375,6 +1985,7 @@ public class FieldTypes {
   }
 
   public synchronized void setDocValuesFormat(String fieldName, String docValuesFormat) {
+    // nocommit can we prevent this, if our codec isn't used by IW?
     try {
       // Will throw exception if this docValuesFormat is unrecognized:
       DocValuesFormat.forName(docValuesFormat);
@@ -1397,7 +2008,7 @@ public class FieldTypes {
     }
   }
 
-  private final Similarity similarity = new PerFieldSimilarityWrapper() {
+  private Similarity similarity = new PerFieldSimilarityWrapper() {
       @Override
       public Similarity get(String fieldName) {
         FieldType field = fields.get(fieldName);
@@ -1414,7 +2025,7 @@ public class FieldTypes {
 
   // nocommit but how can we randomized Codec in tests?
   private final Codec codec = new Lucene50Codec() {
-      // nocommit: too bad we can't just set every format here?  what if we fix this schema to record which format per field, and then
+      // TODO: too bad we can't just set every format here?  what if we fix this schema to record which format per field, and then
       // remove PerFieldXXXFormat...?
       @Override
       public PostingsFormat getPostingsFormatForField(String fieldName) {
@@ -1422,9 +2033,12 @@ public class FieldTypes {
         if (field == null) {
           return super.getPostingsFormatForField(fieldName);
         }
+
         if (field.postingsFormat != null) {
+          // Field has a custom PF:
           return PostingsFormat.forName(field.postingsFormat);
-        } else if (field.blockTreeMinItemsInBlock != null || field.blockTreeMinItemsInAutoPrefix != null) {
+        } else if (field.blockTreeMinItemsInBlock != null || field.fastRanges == Boolean.TRUE) {
+          // Field has the default PF, but we customize BlockTree params:
           int minItemsInBlock, maxItemsInBlock;
           int minItemsInAutoPrefix, maxItemsInAutoPrefix;
           if (field.blockTreeMinItemsInBlock != null) {
@@ -1439,12 +2053,14 @@ public class FieldTypes {
             assert field.blockTreeMaxItemsInAutoPrefix != null;
             minItemsInAutoPrefix = field.blockTreeMinItemsInAutoPrefix.intValue();
             maxItemsInAutoPrefix = field.blockTreeMaxItemsInAutoPrefix.intValue();
-          } else {
+          } else if (field.fastRanges == Boolean.TRUE) {
             minItemsInAutoPrefix = BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE;
             maxItemsInAutoPrefix = BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE;
+          } else {
+            minItemsInAutoPrefix = 0;
+            maxItemsInAutoPrefix = 0;
           }
 
-          // nocommit do we now have cleaner API for this?  Ie "get me default PF, changing these settings"...
           return new Lucene50PostingsFormat(minItemsInBlock, maxItemsInBlock,
                                             minItemsInAutoPrefix, maxItemsInAutoPrefix);
         }
@@ -1461,10 +2077,10 @@ public class FieldTypes {
       }
     };
 
-  private static final Analyzer KEYWORD_ANALYZER = new Analyzer() {
+  private static final Analyzer SINGLE_TOKEN_ANALYZER = new Analyzer() {
       @Override
       protected TokenStreamComponents createComponents(final String fieldName) {
-        return new TokenStreamComponents(new CoreKeywordTokenizer());
+        return new TokenStreamComponents(new SingleTokenTokenizer());
       }
     };
 
@@ -1476,55 +2092,14 @@ public class FieldTypes {
     @Override
     public int getPositionIncrementGap(String fieldName) {
       throw new UnsupportedOperationException();
-
-      /*
-      FieldType field = fields.get(fieldName);
-      if (field == null) {
-        if (defaultIndexAnalyzer == null) {
-          // nocommit sheisty
-          return 0;
-        } else {
-          return defaultIndexAnalyzer.getPositionIncrementGap(fieldName);
-        }
-      }
-      if (field.analyzerPositionGap != null) {
-        return field.analyzerPositionGap.intValue();
-      } else if (field.indexAnalyzer != null) {
-        return field.indexAnalyzer.getPositionIncrementGap(fieldName);
-      } else if (defaultIndexAnalyzer == null) {
-        return 0;
-      } else {
-        return defaultIndexAnalyzer.getPositionIncrementGap(fieldName);
-      }
-      */
     }
 
     @Override
     public int getOffsetGap(String fieldName) {
       throw new UnsupportedOperationException();
-      /*
-      FieldType field = fields.get(fieldName);
-      if (field == null) {
-        if (defaultIndexAnalyzer == null) {
-          // nocommit sheisty
-          return 1;
-        } else {
-          return defaultIndexAnalyzer.getOffsetGap(fieldName);
-        }
-      }
-      if (field.analyzerOffsetGap != null) {
-        return field.analyzerOffsetGap.intValue();
-      } else if (field.indexAnalyzer != null) {
-        return field.indexAnalyzer.getOffsetGap(fieldName);
-      } else if (defaultIndexAnalyzer == null) {
-        return 1;
-      } else {
-        return defaultIndexAnalyzer.getOffsetGap(fieldName);
-      }
-      */
     }
 
-    // nocommit what about wrapReader?
+    // nocommit: what about wrapReader?
   }
 
   private final Analyzer indexAnalyzer = new FieldTypeAnalyzer() {
@@ -1556,7 +2131,7 @@ public class FieldTypes {
         if (field.wrappedQueryAnalyzer != null) {
           return field.wrappedQueryAnalyzer;
         } else if (field.valueType == ValueType.ATOM) {
-          return KEYWORD_ANALYZER;
+          return SINGLE_TOKEN_ANALYZER;
         }
         return defaultQueryAnalyzer;
       }
@@ -1590,7 +2165,7 @@ public class FieldTypes {
     return queryAnalyzer;
   }
 
-  // nocommit we should note that the field has a specific analyzer set, and then throw exc if it didn't get set again after load
+  // TODO: we could note that the field had a specific analyzer set, and then throw exc if it didn't get set again after load?
 
   /** NOTE: analyzer does not persist, so each time you create {@code FieldTypes} from
    *  {@linkIndexWriter} or {@link IndexReader} you must set all per-field analyzers again. */
@@ -1606,6 +2181,7 @@ public class FieldTypes {
     if (current == null) {
       current = newFieldType(fieldName);
       current.indexAnalyzer = analyzer;
+      current.reWrapAnalyzers(defaultIndexAnalyzer, defaultQueryAnalyzer);
       fields.put(fieldName, current);
       changed();
     } else if (current.indexAnalyzer == null) {
@@ -1622,10 +2198,26 @@ public class FieldTypes {
       current.reWrapAnalyzers(defaultIndexAnalyzer, defaultQueryAnalyzer);
       changed();
     } else {
-      illegalState(fieldName, "analyzer was already set");
+      illegalState(fieldName, "indexAnalyzer was already set");
     }
   }
 
+  public synchronized void setDefaultSimilarity(Similarity sim) {
+    this.defaultSimilarity = sim;
+  }
+
+  public synchronized Similarity getDefaultSimilarity() {
+    return defaultSimilarity;
+  }
+
+  public synchronized void setDefaultQueryAnalyzer(Analyzer a) {
+    this.defaultQueryAnalyzer = a;
+  }
+
+  public synchronized Analyzer getDefaultQueryAnalyzer() {
+    return defaultQueryAnalyzer;
+  }
+
   public synchronized Analyzer getIndexAnalyzer(String fieldName) {
     return getFieldType(fieldName).indexAnalyzer;
   }
@@ -1653,7 +2245,7 @@ public class FieldTypes {
       current.reWrapAnalyzers(defaultIndexAnalyzer, defaultQueryAnalyzer);
       changed();
     } else {
-      illegalState(fieldName, "analyzer was already set");
+      illegalState(fieldName, "queryAnalyzer was already set");
     }
   }
 
@@ -1662,7 +2254,7 @@ public class FieldTypes {
   }
 
   /** NOTE: similarity does not persist, so each time you create {@code FieldTypes} from
-   *  {@linkIndexWriter} or {@link IndexReader} you must set all per-field similarities again. */
+   *  {@linkIndexWriter} or {@link IndexReader} you must set all per-field similarities again.  This can be changed at any time. */
   public synchronized void setSimilarity(String fieldName, Similarity similarity) {
     FieldType current = fields.get(fieldName);
     if (current == null) {
@@ -1671,7 +2263,6 @@ public class FieldTypes {
       fields.put(fieldName, current);
       changed();
     } else {
-      // nocommit should we not allow this...
       current.similarity = similarity;
       changed();
     }
@@ -1725,8 +2316,7 @@ public class FieldTypes {
     } else if (current.minTokenLength == null ||
                current.minTokenLength.intValue() != minTokenLength ||
                current.maxTokenLength.intValue() != maxTokenLength) {
-      Integer oldMin = current.minTokenLength;
-      Integer oldMax = current.maxTokenLength;
+      FieldType sav = new FieldType(current);
       boolean success = false;
       try {
         current.minTokenLength = minTokenLength;
@@ -1735,8 +2325,7 @@ public class FieldTypes {
         success = true;
       } finally {
         if (success == false) {
-          current.minTokenLength = oldMin;
-          current.maxTokenLength = oldMax;
+          fields.put(fieldName, current);
         }
       }
       current.reWrapAnalyzers(defaultIndexAnalyzer, defaultQueryAnalyzer);
@@ -1744,8 +2333,6 @@ public class FieldTypes {
     }
   }
 
-  // nocommit clearMinMaxTokenLength
-
   public synchronized Integer getMinTokenLength(String fieldName) {
     return getFieldType(fieldName).minTokenLength;
   }
@@ -1758,8 +2345,6 @@ public class FieldTypes {
     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);
@@ -1856,6 +2441,7 @@ public class FieldTypes {
    *  effect for newly written (flushed or merged) segments.  The default is 25; higher values make fewer, larger blocks, which require less
    *  heap in the IndexReader but slows down term lookups. */
   public synchronized void setTermsDictBlockSize(String fieldName, int minItemsInBlock) {
+    // nocommit can we prevent this, if our codec isn't used by IW?
     setTermsDictBlockSize(fieldName, minItemsInBlock, 2*(minItemsInBlock-1));
   }
 
@@ -1863,6 +2449,7 @@ public class FieldTypes {
    *  effect for newly written (flushed or merged) segments.  The default is 25 and 48; higher values make fewer, larger blocks, which require less
    *  heap in the IndexReader but slows down term lookups. */
   public synchronized void setTermsDictBlockSize(String fieldName, int minItemsInBlock, int maxItemsInBlock) {
+    // nocommit can we prevent this, if our codec isn't used by IW?
     ensureWritable();
 
     // nocommit must check that field is in fact using block tree?
@@ -1897,6 +2484,7 @@ public class FieldTypes {
     } else {
       current.blockTreeMinItemsInBlock = minItemsInBlock;
       current.blockTreeMaxItemsInBlock = maxItemsInBlock;
+      changed();
       assert current.validate();
     }
   }
@@ -1945,6 +2533,7 @@ public class FieldTypes {
     } else {
       current.blockTreeMinItemsInAutoPrefix = minItemsInAutoPrefix;
       current.blockTreeMaxItemsInAutoPrefix = maxItemsInAutoPrefix;
+      changed();
       assert current.validate();
     }
   }
@@ -1995,8 +2584,6 @@ public class FieldTypes {
       changed();
     } else if (current.sortable != Boolean.FALSE) {
       // nocommit don't we need to ... turn off DocValues if they were only on because of sorting?
-      // nocommit ok to allow this?
-      // nocommit should we validate?
       current.sortable = Boolean.FALSE;
       current.sortReversed = null;
       changed();
@@ -2007,6 +2594,83 @@ public class FieldTypes {
     return getFieldType(fieldName).sortable == Boolean.TRUE;
   }
 
+  private boolean isNumericType(ValueType type) {
+    return type == ValueType.INT ||
+      type == ValueType.HALF_FLOAT ||
+      type == ValueType.FLOAT ||
+      type == ValueType.LONG ||
+      type == ValueType.DOUBLE ||
+      type == ValueType.DATE;
+  }
+
+  /** For multi-valued numeric fields, sets which value should be selected for sorting.  This can be changed at any time. */
+  public synchronized void setMultiValuedNumericSortSelector(String fieldName, SortedNumericSelector.Type selector) {
+    // field must exist
+    FieldType current = getFieldType(fieldName);
+    if (current.multiValued != Boolean.TRUE) {
+      illegalState(fieldName, "this field is not multi-valued");
+    }
+    if (isNumericType(current.valueType) == false) {
+      illegalState(fieldName, "value type must be INT, HALF_FLOAT, FLOAT, LONG, DOUBLE or DATE; got value type=" + current.valueType);
+    }
+    if (current.sortable != Boolean.TRUE) {
+      illegalState(fieldName, "field is not enabled for sorting");
+    }
+    if (current.numericSelector != selector) {
+      current.numericSelector = selector;
+      changed(false);
+    }
+  }
+
+  public synchronized SortedNumericSelector.Type getMultiValuedNumericSortSelector(String fieldName) {
+    FieldType current = getFieldType(fieldName);
+    if (current.multiValued != Boolean.TRUE) {
+      illegalState(fieldName, "this field is not multi-valued");
+    }
+    if (isNumericType(current.valueType) == false) {
+      illegalState(fieldName, "value type must be INT, HALF_FLOAT, FLOAT, LONG, DOUBLE or DATE; got value type=" + current.valueType);
+    }
+    if (current.sortable != Boolean.TRUE) {
+      illegalState(fieldName, "field is not enabled for sorting");
+    }
+    return current.numericSelector;
+  }
+
+  /** For multi-valued binary fields, sets which value should be selected for sorting.  This can be changed at any time. */
+  public synchronized void setMultiValuedStringSortSelector(String fieldName, SortedSetSelector.Type selector) {
+    // field must exist
+    FieldType current = getFieldType(fieldName);
+    if (current.multiValued != Boolean.TRUE) {
+      illegalState(fieldName, "this field is not multi-valued");
+    }
+    if (current.valueType != ValueType.BIG_INT &&
+        current.valueType != ValueType.ATOM) {
+      illegalState(fieldName, "value type must be BIG_INT or ATOM; got value type=" + current.valueType);
+    }
+    if (current.sortable != Boolean.TRUE) {
+      illegalState(fieldName, "field is not enabled for sorting");
+    }
+    if (current.sortedSetSelector != selector) {
+      current.sortedSetSelector = selector;
+      changed(false);
+    }
+  }
+
+  public synchronized SortedSetSelector.Type getMultiValuedStringSortSelector(String fieldName) {
+    FieldType current = getFieldType(fieldName);
+    if (current.multiValued != Boolean.TRUE) {
+      illegalState(fieldName, "this field is not multi-valued");
+    }
+    if (current.valueType != ValueType.BIG_INT &&
+        current.valueType != ValueType.ATOM) {
+      illegalState(fieldName, "value type must be BIG_INT or ATOM; got value type=" + current.valueType);
+    }
+    if (current.sortable != Boolean.TRUE) {
+      illegalState(fieldName, "field is not enabled for sorting");
+    }
+    return current.sortedSetSelector;
+  }
+
   public synchronized void setSortMissingFirst(String fieldName) {
     FieldType current = fields.get(fieldName);
     if (current == null) {
@@ -2027,7 +2691,7 @@ public class FieldTypes {
             current.sortMissingLast = currentValue;
           }
         }
-        changed();
+        changed(false);
       }
     }
   }
@@ -2052,7 +2716,7 @@ public class FieldTypes {
             current.sortMissingLast = currentValue;
           }
         }
-        changed();
+        changed(false);
       }
     }
   }
@@ -2082,7 +2746,7 @@ public class FieldTypes {
     }
   }
 
-  /** Disables fast range filters for this field. */
+  /** Disables fast range filters for this field.  You can do this at any time, but once it's disabled you cannot re-enable it. */
   public synchronized void disableFastRanges(String fieldName) {
     FieldType current = fields.get(fieldName);
     if (current == null) {
@@ -2091,8 +2755,6 @@ public class FieldTypes {
       fields.put(fieldName, current);
       changed();
     } else if (current.fastRanges != Boolean.FALSE) {
-      // nocommit ok to allow this?
-      // nocommit should we validate?
       current.fastRanges = Boolean.FALSE;
       changed();
     }
@@ -2123,7 +2785,7 @@ public class FieldTypes {
       }
       changed();
     } else if (current.highlighted == Boolean.FALSE) {
-      illegalState(fieldName, "cannot enable highlighting: it was already disabled");
+      illegalState(fieldName, "highlighting was already disabled");
     }
   }
 
@@ -2135,7 +2797,7 @@ public class FieldTypes {
       current.highlighted = Boolean.FALSE;
       fields.put(fieldName, current);
       changed();
-    } else if (current.highlighted == null) {
+    } else if (current.highlighted != Boolean.TRUE) {
       Boolean currentValue = current.highlighted;
       boolean success = false;
       try {
@@ -2176,7 +2838,7 @@ public class FieldTypes {
       }
       changed();
     } else if (current.indexNorms == Boolean.FALSE) {
-      illegalState(fieldName, "cannot enable norms that were already disable");
+      illegalState(fieldName, "norms were already disable");
     }
   }
 
@@ -2198,43 +2860,115 @@ public class FieldTypes {
     return getFieldType(fieldName).indexNorms == Boolean.TRUE;
   }
 
-  /** Store values for this field.  This can be changed at any time. */
-  public synchronized void enableStored(String fieldName) {
-    FieldType current = fields.get(fieldName);
-    if (current == null) {
-      current = newFieldType(fieldName);
-      current.stored = Boolean.TRUE;
-      fields.put(fieldName, current);
-      changed();
-    } else if (current.stored != Boolean.TRUE) {
-      // nocommit should this change not be allowed...
-      current.stored = Boolean.TRUE;
-      changed();
+  /** Sets the maximum precision for this big int field. */
+  public void setBigIntByteWidth(String fieldName, int bytes) {
+    if (bytes <= 0) {
+      illegalState(fieldName, "bytes must be > 0; got: " + bytes);
     }
-  }
 
-  /** Do not store values for this field.  This can be changed at any time. */
-  public synchronized void disableStored(String fieldName) {
     FieldType current = fields.get(fieldName);
     if (current == null) {
       current = newFieldType(fieldName);
-      current.stored = Boolean.FALSE;
+      current.storedOnly = Boolean.FALSE;
+      current.valueType = ValueType.BIG_INT;
+      current.bigIntByteWidth = bytes;
       fields.put(fieldName, current);
+      current.setDefaults();
       changed();
-    } else if (current.stored == null || current.stored == Boolean.TRUE) {
-      // nocommit should this change not be allowed...
-      current.stored = Boolean.FALSE;
+    } else if (current.valueType == ValueType.NONE) {
+      current.storedOnly = Boolean.FALSE;
+      current.valueType = ValueType.BIG_INT;
+      current.bigIntByteWidth = bytes;
+      current.setDefaults();
+      changed();
+    } else if (current.valueType != ValueType.BIG_INT) {
+      illegalState(fieldName, "can only setBigIntByteWidth on BIG_INT fields; got value type=" + current.valueType);
+    } else if (current.bigIntByteWidth == null) {
+      current.bigIntByteWidth = bytes;
       changed();
+    } else if (current.bigIntByteWidth.intValue() != bytes) {
+      illegalState(fieldName, "cannot change bigIntByteWidth from " + current.bigIntByteWidth + " to " + bytes);
     }
   }
 
-  /** Whether this field's value is stored. */
-  public synchronized boolean getStored(String fieldName) {
-    return getFieldType(fieldName).stored == Boolean.TRUE;
+  public int getBigIntByteWidth(String fieldName) {
+    // field must exist
+    FieldType current = getFieldType(fieldName);
+    if (current.valueType != ValueType.BIG_INT) {
+      illegalState(fieldName, "field is not BIG_INT; got value type=" + current.valueType);
+    }
+    return current.bigIntByteWidth;
   }
 
-  // nocommit should we make a single method to enable the different combinations...?
-  public synchronized void enableTermVectors(String fieldName) {
+  /** All indexed terms are reversed before indexing, using {@code ReverseStringFilter}.  This requires that Lucene's analysis module is on
+   *  the classpath. */
+  public void setReversedTerms(String fieldName) {
+    FieldType current = fields.get(fieldName);
+    if (current == null) {
+      current = newFieldType(fieldName);
+      current.reversedTerms = true;
+      fields.put(fieldName, current);
+      current.reWrapAnalyzers(defaultIndexAnalyzer, defaultQueryAnalyzer);
+      changed();
+    } else if (current.reversedTerms == null) {
+      current.reversedTerms = true;
+      boolean success = false;
+      try {
+        current.validate();
+        success = true;
+      } finally {
+        if (success == false) {
+          current.reversedTerms = null;
+        }
+      }
+      current.reWrapAnalyzers(defaultIndexAnalyzer, defaultQueryAnalyzer);
+      changed();
+    } else if (current.reversedTerms != Boolean.TRUE) {
+      illegalState(fieldName, "can only setReversedTerms before the field is indexed");
+    }
+  }
+
+  public Boolean getReversedTerms(String fieldName) {
+    // field must exist
+    FieldType current = getFieldType(fieldName);
+    return current.reversedTerms; 
+  }
+
+  /** Store values for this field.  This can be changed at any time. */
+  public synchronized void enableStored(String fieldName) {
+    FieldType current = fields.get(fieldName);
+    if (current == null) {
+      current = newFieldType(fieldName);
+      current.stored = Boolean.TRUE;
+      fields.put(fieldName, current);
+      changed();
+    } else if (current.stored != Boolean.TRUE) {
+      current.stored = Boolean.TRUE;
+      changed();
+    }
+  }
+
+  /** Do not store values for this field.  This can be changed at any time. */
+  public synchronized void disableStored(String fieldName) {
+    FieldType current = fields.get(fieldName);
+    if (current == null) {
+      current = newFieldType(fieldName);
+      current.stored = Boolean.FALSE;
+      fields.put(fieldName, current);
+      changed();
+    } else if (current.stored == null || current.stored == Boolean.TRUE) {
+      current.stored = Boolean.FALSE;
+      changed();
+    }
+  }
+
+  /** Whether this field's value is stored. */
+  public synchronized boolean getStored(String fieldName) {
+    return getFieldType(fieldName).stored == Boolean.TRUE;
+  }
+
+  /** Enable term vectors for this field.  This can be changed at any time. */
+  public synchronized void enableTermVectors(String fieldName) {
     FieldType current = fields.get(fieldName);
     if (current == null) {
       current = newFieldType(fieldName);
@@ -2242,12 +2976,12 @@ public class FieldTypes {
       fields.put(fieldName, current);
       changed();
     } else if (current.storeTermVectors != Boolean.TRUE) {
-      // nocommit should this change not be allowed...
       current.storeTermVectors = Boolean.TRUE;
       changed();
     }
   }
 
+  /** Disable term vectors for this field.  This can be changed at any time. */
   public synchronized void disableTermVectors(String fieldName) {
     FieldType current = fields.get(fieldName);
     if (current == null) {
@@ -2256,7 +2990,6 @@ public class FieldTypes {
       fields.put(fieldName, current);
       changed();
     } else if (current.storeTermVectors != Boolean.FALSE) {
-      // nocommit should this change not be allowed...
       current.storeTermVectors = Boolean.FALSE;
       changed();
     }
@@ -2266,23 +2999,22 @@ public class FieldTypes {
     return getFieldType(fieldName).storeTermVectors == Boolean.TRUE;
   }
 
+  /** Enable term vector offsets for this field.  This can be changed at any time. */
   public synchronized void enableTermVectorOffsets(String fieldName) {
     FieldType current = fields.get(fieldName);
     if (current == null || current.storeTermVectors != Boolean.TRUE) {
-      // nocommit we could enable term vectors for you?
       illegalState(fieldName, "cannot enable termVectorOffsets when termVectors haven't been enabled");
     }
     if (current.storeTermVectorOffsets != Boolean.TRUE) {
-      // nocommit should this change not be allowed...
       current.storeTermVectorOffsets = Boolean.TRUE;
       changed();
     }
   }
 
+  /** Disable term vector offsets for this field.  This can be changed at any time. */
   public synchronized void disableTermVectorOffsets(String fieldName) {
     FieldType current = getFieldType(fieldName);
     if (current.storeTermVectorOffsets == Boolean.TRUE) {
-      // nocommit should this change not be allowed...
       current.storeTermVectorOffsets = Boolean.FALSE;
       changed();
     }
@@ -2292,23 +3024,22 @@ public class FieldTypes {
     return getFieldType(fieldName).storeTermVectorOffsets == Boolean.TRUE;
   }
 
+  /** Enable term vector positions for this field.  This can be changed at any time. */
   public synchronized void enableTermVectorPositions(String fieldName) {
     FieldType current = fields.get(fieldName);
     if (current == null || current.storeTermVectors != Boolean.TRUE) {
-      // nocommit we could enable term vectors for you?
       illegalState(fieldName, "cannot enable termVectorPositions when termVectors haven't been enabled");
     }
     if (current.storeTermVectorPositions != Boolean.TRUE) {
-      // nocommit should this change not be allowed...
       current.storeTermVectorPositions = Boolean.TRUE;
       changed();
     }
   }
 
+  /** Disable term vector positions for this field.  This can be changed at any time. */
   public synchronized void disableTermVectorPositions(String fieldName) {
     FieldType current = getFieldType(fieldName);
     if (current.storeTermVectorPositions == Boolean.TRUE) {
-      // nocommit should this change not be allowed...
       current.storeTermVectorPositions = Boolean.FALSE;
       changed();
     }
@@ -2318,26 +3049,25 @@ public class FieldTypes {
     return getFieldType(fieldName).storeTermVectorPositions == Boolean.TRUE;
   }
 
+  /** Enable term vector payloads for this field.  This can be changed at any time. */
   public synchronized void enableTermVectorPayloads(String fieldName) {
     FieldType current = fields.get(fieldName);
     if (current == null || current.storeTermVectors != Boolean.TRUE) {
-      // nocommit we could enable term vectors / positions for you?
       illegalState(fieldName, "cannot enable termVectorPayloads when termVectors haven't been enabled");
     }
     if (current.storeTermVectorPositions != Boolean.TRUE) {
       illegalState(fieldName, "cannot enable termVectorPayloads when termVectorPositions haven't been enabled");
     }
     if (current.storeTermVectorPayloads != Boolean.TRUE) {
-      // nocommit should this change not be allowed...
       current.storeTermVectorPayloads = Boolean.TRUE;
       changed();
     }
   }
 
+  /** Disable term vector payloads for this field.  This can be changed at any time. */
   public synchronized void disableTermVectorPayloads(String fieldName) {
     FieldType current = getFieldType(fieldName);
     if (current.storeTermVectorPayloads == Boolean.TRUE) {
-      // nocommit should this change not be allowed...
       current.storeTermVectorPayloads = Boolean.FALSE;
       changed();
     }
@@ -2390,14 +3120,8 @@ public class FieldTypes {
   }
 
   public synchronized IndexOptions getIndexOptions(String fieldName) {
-    // nocommit throw exc if field doesn't exist?
-    FieldType current = fields.get(fieldName);
-    if (current == null) {
-      // nocommit IO.NONE?
-      return null;
-    } else {
-      return current.indexOptions;
-    }
+    // Field must exist:
+    return getFieldType(fieldName).indexOptions;
   }
 
   public synchronized void disableDocValues(String fieldName) {
@@ -2437,14 +3161,8 @@ public class FieldTypes {
   }
 
   public synchronized DocValuesType getDocValuesType(String fieldName, DocValuesType dvType) {
-    // nocommit should we insist field exists?
-    FieldType current = fields.get(fieldName);
-    if (current == null) {
-      // nocommit dvt.NONE?
-      return null;
-    } else {
-      return current.docValuesType;
-    }
+    // Field must exist:
+    return getFieldType(fieldName).docValuesType;
   }
 
   synchronized void recordValueType(String fieldName, ValueType valueType) {
@@ -2456,36 +3174,141 @@ public class FieldTypes {
     indexedDocs = true;
     FieldType current = fields.get(fieldName);
     if (current == null) {
+      if (valueType == ValueType.BIG_INT) {
+        illegalState(fieldName, "you must first set the byte width for big_int fields");
+      }
       current = newFieldType(fieldName);
+      current.storedOnly = Boolean.FALSE;
       current.valueType = valueType;
       current.isUnique = isUnique;
       fields.put(fieldName, current);
-      setDefaults(current);
+      current.setDefaults();
       changed();
     } else if (current.valueType == ValueType.NONE) {
+      if (valueType == ValueType.BIG_INT) {
+        illegalState(fieldName, "you must first set the byte width for big_int fields");
+      }
+
       if (current.isUnique != null && current.isUnique.booleanValue() != isUnique) {
-        // nocommit make sure test covers this
         illegalState(fieldName, "cannot change to isUnique to " + isUnique + ": field was already with isUnique=" + current.isUnique);
       }
 
-      Boolean currentIsUnique = current.isUnique;
-      // This can happen if e.g. the app first calls FieldTypes.setStored(...)
+      FieldType sav = new FieldType(current);
+      // This can happen if e.g. the app first calls FieldTypes.enableStored(...)
       boolean success = false;
       try {
         current.isUnique = isUnique;
+        assert current.storedOnly == null;
+        current.storedOnly = Boolean.FALSE;
         current.valueType = valueType;
+        current.setDefaults();
         current.validate();
         success = true;
       } finally {
         if (success == false) {
-          current.valueType = ValueType.NONE;
-          current.isUnique = currentIsUnique;
+          fields.put(fieldName, sav);
         }
       }
-      setDefaults(current);
+      current.setDefaults();
       changed();
     } else if (current.valueType != valueType) {
       illegalState(fieldName, "cannot change from value type " + current.valueType + " to " + valueType);
+    } else if (current.storedOnly == Boolean.TRUE) {
+      illegalState(fieldName, "this field is only stored; use addStoredXXX instead");
+    } else if (current.isUnique == null) {
+      current.isUnique = isUnique;
+      changed();
+    } else if (current.isUnique != isUnique) {
+      illegalState(fieldName, "cannot change isUnique from " + current.isUnique + " to " + isUnique);
+    }
+  }
+
+  synchronized void recordStringAtomValueType(String fieldName, boolean isUnique) {
+    ensureWritable();
+    indexedDocs = true;
+    FieldType current = fields.get(fieldName);
+    if (current == null) {
+      current = newFieldType(fieldName);
+      current.storedOnly = Boolean.FALSE;
+      current.valueType = ValueType.ATOM;
+      current.isBinary = Boolean.FALSE;
+      current.isUnique = isUnique;
+      fields.put(fieldName, current);
+      current.setDefaults();
+      changed();
+    } else if (current.valueType == ValueType.NONE) {
+      if (current.isUnique != null && current.isUnique.booleanValue() != isUnique) {
+        illegalState(fieldName, "cannot change to isUnique to " + isUnique + ": field was already with isUnique=" + current.isUnique);
+      }
+
+      FieldType sav = new FieldType(current);
+      // This can happen if e.g. the app first calls FieldTypes.enableStored(...)
+      boolean success = false;
+      try {
+        current.isBinary = Boolean.FALSE;
+        current.isUnique = isUnique;
+        assert current.storedOnly == null;
+        current.storedOnly = Boolean.FALSE;
+        current.valueType = ValueType.ATOM;
+        current.setDefaults();
+        current.validate();
+        success = true;
+      } finally {
+        if (success == false) {
+          fields.put(fieldName, sav);
+        }
+      }
+      changed();
+    } else if (current.valueType != ValueType.ATOM) {
+      illegalState(fieldName, "cannot change from value type " + current.valueType + " to ATOM");
+    } else if (current.isBinary != Boolean.FALSE) {
+      illegalState(fieldName, "cannot change from binary to non-binary ATOM");
+    } else if (current.isUnique != isUnique) {
+      illegalState(fieldName, "cannot change isUnique from " + current.isUnique + " to " + isUnique);
+    }
+  }
+
+  synchronized void recordBinaryAtomValueType(String fieldName, boolean isUnique) {
+    ensureWritable();
+    indexedDocs = true;
+    FieldType current = fields.get(fieldName);
+    if (current == null) {
+      current = newFieldType(fieldName);
+      current.storedOnly = Boolean.FALSE;
+      current.valueType = ValueType.ATOM;
+      current.isBinary = Boolean.TRUE;
+      current.isUnique = isUnique;
+      fields.put(fieldName, current);
+      current.setDefaults();
+      changed();
+    } else if (current.valueType == ValueType.NONE) {
+      if (current.isUnique != null && current.isUnique.booleanValue() != isUnique) {
+        illegalState(fieldName, "cannot change to isUnique to " + isUnique + ": field was already with isUnique=" + current.isUnique);
+      }
+
+      FieldType sav = new FieldType(current);
+      // This can happen if e.g. the app first calls FieldTypes.enableStored(...)
+      boolean success = false;
+      try {
+        current.isBinary = Boolean.TRUE;
+        current.isUnique = isUnique;
+        assert current.storedOnly == null;
+        current.storedOnly = Boolean.FALSE;
+        current.valueType = ValueType.ATOM;
+        current.setDefaults();
+        current.validate();
+        success = true;
+      } finally {
+        if (success == false) {
+          fields.put(fieldName, sav);
+        }
+      }
+      current.setDefaults();
+      changed();
+    } else if (current.valueType != ValueType.ATOM) {
+      illegalState(fieldName, "cannot change from value type " + current.valueType + " to ATOM");
+    } else if (current.isBinary != Boolean.TRUE) {
+      illegalState(fieldName, "cannot change from string to binary ATOM");
     } else if (current.isUnique != isUnique) {
       illegalState(fieldName, "cannot change isUnique from " + current.isUnique + " to " + isUnique);
     }
@@ -2497,6 +3320,7 @@ public class FieldTypes {
     FieldType current = fields.get(fieldName);
     if (current == null) {
       current = newFieldType(fieldName);
+      current.storedOnly = Boolean.TRUE;
       current.valueType = valueType;
       current.isUnique = Boolean.FALSE;
       current.indexOptionsSet = true;
@@ -2504,39 +3328,46 @@ public class FieldTypes {
       current.docValuesTypeSet = true;
       current.docValuesType = DocValuesType.NONE;
       fields.put(fieldName, current);
-      setDefaults(current);
+      current.setDefaults();
       changed();
-    } else {
+
+    } else if (current.storedOnly == Boolean.FALSE) {
+      illegalState(fieldName, "cannot addStored: field was already added non-stored");
+    } else if (current.storedOnly == null) {
 
       if (current.indexOptionsSet && current.indexOptions != IndexOptions.NONE) {
-        // nocommit testme
         illegalState(fieldName, "cannot addStored: field is already indexed with indexOptions=" + current.indexOptions);
       }
 
+      // nocommit why not?
+      /*
       if (current.docValuesTypeSet && current.docValuesType != DocValuesType.NONE) {
-        // nocommit testme
         illegalState(fieldName, "cannot addStored: field already has docValuesType=" + current.docValuesType);
       }
+      */
 
-      if (current.valueType == ValueType.NONE) {
-        FieldType sav = new FieldType(current);
-        boolean success = false;
-        try {
-          current.valueType = valueType;
-          current.indexOptions = IndexOptions.NONE;
+      // All methods that set valueType also set storedOnly to false:
+      assert current.valueType == ValueType.NONE;
+
+      FieldType sav = new FieldType(current);
+      boolean success = false;
+      try {
+        current.storedOnly = Boolean.TRUE;
+        current.valueType = valueType;
+        current.indexOptions = IndexOptions.NONE;
+        if (current.docValuesTypeSet == false) {
           current.docValuesType = DocValuesType.NONE;
-          current.validate();
-          success = true;
-        } finally {
-          if (success == false) {

[... 1226 lines stripped ...]