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/10/31 17:17:29 UTC

svn commit: r1635804 [2/5] - in /lucene/dev/branches/branch_5x: ./ lucene/ lucene/backward-codecs/ lucene/backward-codecs/src/java/org/apache/lucene/codecs/blocktree/ lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/ lucene/backward-co...

Modified: lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java Fri Oct 31 16:17:25 2014
@@ -29,9 +29,9 @@ import org.apache.lucene.codecs.Postings
 import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat; // javadocs
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.OrdTermState;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
@@ -327,7 +327,7 @@ public final class DirectPostingsFormat 
 
       this.minSkipCount = minSkipCount;
 
-      hasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_ONLY) > 0;
+      hasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS) > 0;
       hasPos = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) > 0;
       hasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) > 0;
       hasPayloads = fieldInfo.hasPayloads();

Modified: lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsReader.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsReader.java (original)
+++ lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsReader.java Fri Oct 31 16:17:25 2014
@@ -29,7 +29,7 @@ import java.util.TreeMap;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
@@ -110,7 +110,7 @@ public class FSTOrdTermsReader extends F
       final int numFields = blockIn.readVInt();
       for (int i = 0; i < numFields; i++) {
         FieldInfo fieldInfo = fieldInfos.fieldInfo(blockIn.readVInt());
-        boolean hasFreq = fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY;
+        boolean hasFreq = fieldInfo.getIndexOptions() != IndexOptions.DOCS;
         long numTerms = blockIn.readVLong();
         long sumTotalTermFreq = hasFreq ? blockIn.readVLong() : -1;
         long sumDocFreq = blockIn.readVLong();

Modified: lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java Fri Oct 31 16:17:25 2014
@@ -25,11 +25,11 @@ import org.apache.lucene.codecs.BlockTer
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.PostingsWriterBase;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
@@ -231,7 +231,7 @@ public class FSTOrdTermsWriter extends F
         for (FieldMetaData field : fields) {
           blockOut.writeVInt(field.fieldInfo.number);
           blockOut.writeVLong(field.numTerms);
-          if (field.fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
+          if (field.fieldInfo.getIndexOptions() != IndexOptions.DOCS) {
             blockOut.writeVLong(field.sumTotalTermFreq);
           }
           blockOut.writeVLong(field.sumDocFreq);
@@ -335,7 +335,7 @@ public class FSTOrdTermsWriter extends F
         if (delta == 0) {
           statsOut.writeVInt(state.docFreq<<1|1);
         } else {
-          statsOut.writeVInt(state.docFreq<<1|0);
+          statsOut.writeVInt(state.docFreq<<1);
           statsOut.writeVLong(state.totalTermFreq-state.docFreq);
         }
       } else {

Modified: lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermOutputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermOutputs.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermOutputs.java (original)
+++ lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermOutputs.java Fri Oct 31 16:17:25 2014
@@ -22,7 +22,7 @@ import java.util.Arrays;
 import java.util.Collections;
 
 import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.util.Accountable;
@@ -129,7 +129,7 @@ class FSTTermOutputs extends Outputs<FST
   }
   
   protected FSTTermOutputs(FieldInfo fieldInfo, int longsSize) {
-    this.hasPos = (fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY);
+    this.hasPos = fieldInfo.getIndexOptions() != IndexOptions.DOCS;
     this.longsSize = longsSize;
   }
 

Modified: lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsReader.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsReader.java (original)
+++ lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsReader.java Fri Oct 31 16:17:25 2014
@@ -28,7 +28,7 @@ import java.util.TreeMap;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
@@ -95,7 +95,7 @@ public class FSTTermsReader extends Fiel
         int fieldNumber = in.readVInt();
         FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldNumber);
         long numTerms = in.readVLong();
-        long sumTotalTermFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY ? -1 : in.readVLong();
+        long sumTotalTermFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS ? -1 : in.readVLong();
         long sumDocFreq = in.readVLong();
         int docCount = in.readVInt();
         int longsSize = in.readVInt();

Modified: lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java Fri Oct 31 16:17:25 2014
@@ -25,7 +25,7 @@ import org.apache.lucene.codecs.BlockTer
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.PostingsWriterBase;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.Fields;
@@ -204,7 +204,7 @@ public class FSTTermsWriter extends Fiel
         for (FieldMetaData field : fields) {
           out.writeVInt(field.fieldInfo.number);
           out.writeVLong(field.numTerms);
-          if (field.fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
+          if (field.fieldInfo.getIndexOptions() != IndexOptions.DOCS) {
             out.writeVLong(field.sumTotalTermFreq);
           }
           out.writeVLong(field.sumDocFreq);

Modified: lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java Fri Oct 31 16:17:25 2014
@@ -33,7 +33,7 @@ import org.apache.lucene.index.CorruptIn
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexFileNames;
@@ -145,7 +145,7 @@ public final class MemoryPostingsFormat 
         lastDocID = docID;
         docCount++;
 
-        if (field.getIndexOptions() == IndexOptions.DOCS_ONLY) {
+        if (field.getIndexOptions() == IndexOptions.DOCS) {
           buffer.writeVInt(delta);
         } else if (termDocFreq == 1) {
           buffer.writeVInt((delta<<1) | 1);
@@ -232,7 +232,7 @@ public final class MemoryPostingsFormat 
       assert buffer2.getFilePointer() == 0;
 
       buffer2.writeVInt(stats.docFreq);
-      if (field.getIndexOptions() != IndexOptions.DOCS_ONLY) {
+      if (field.getIndexOptions() != IndexOptions.DOCS) {
         buffer2.writeVLong(stats.totalTermFreq-stats.docFreq);
       }
       int pos = (int) buffer2.getFilePointer();
@@ -262,7 +262,7 @@ public final class MemoryPostingsFormat 
       if (termCount > 0) {
         out.writeVInt(termCount);
         out.writeVInt(field.number);
-        if (field.getIndexOptions() != IndexOptions.DOCS_ONLY) {
+        if (field.getIndexOptions() != IndexOptions.DOCS) {
           out.writeVLong(sumTotalTermFreq);
         }
         out.writeVLong(sumDocFreq);
@@ -470,7 +470,7 @@ public final class MemoryPostingsFormat 
           return docID = NO_MORE_DOCS;
         }
         docUpto++;
-        if (indexOptions == IndexOptions.DOCS_ONLY) {
+        if (indexOptions == IndexOptions.DOCS) {
           accum += in.readVInt();
         } else {
           final int code = in.readVInt();
@@ -754,7 +754,7 @@ public final class MemoryPostingsFormat 
       if (!didDecode) {
         buffer.reset(current.output.bytes, current.output.offset, current.output.length);
         docFreq = buffer.readVInt();
-        if (field.getIndexOptions() != IndexOptions.DOCS_ONLY) {
+        if (field.getIndexOptions() != IndexOptions.DOCS) {
           totalTermFreq = docFreq + buffer.readVLong();
         } else {
           totalTermFreq = -1;
@@ -896,7 +896,7 @@ public final class MemoryPostingsFormat 
       field = fieldInfos.fieldInfo(fieldNumber);
       if (field == null) {
         throw new CorruptIndexException("invalid field number: " + fieldNumber, in);
-      } else if (field.getIndexOptions() != IndexOptions.DOCS_ONLY) {
+      } else if (field.getIndexOptions() != IndexOptions.DOCS) {
         sumTotalTermFreq = in.readVLong();
       } else {
         sumTotalTermFreq = -1;

Modified: lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java (original)
+++ lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java Fri Oct 31 16:17:25 2014
@@ -17,16 +17,6 @@ package org.apache.lucene.codecs.simplet
  * limitations under the License.
  */
 
-import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.END;
-import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.FIELD;
-import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.LENGTH;
-import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.MAXLENGTH;
-import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.MINVALUE;
-import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.NUMVALUES;
-import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.ORDPATTERN;
-import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.PATTERN;
-import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.TYPE;
-
 import java.io.IOException;
 import java.math.BigDecimal;
 import java.math.BigInteger;
@@ -43,8 +33,8 @@ import org.apache.lucene.codecs.DocValue
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SegmentReadState;
@@ -61,6 +51,16 @@ import org.apache.lucene.util.BytesRefBu
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.StringHelper;
 
+import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.END;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.FIELD;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.LENGTH;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.MAXLENGTH;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.MINVALUE;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.NUMVALUES;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.ORDPATTERN;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.PATTERN;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.TYPE;
+
 class SimpleTextDocValuesReader extends DocValuesProducer {
 
   private static final long BASE_RAM_BYTES_USED =
@@ -104,7 +104,7 @@ class SimpleTextDocValuesReader extends 
       assert startsWith(TYPE) : scratch.get().utf8ToString();
 
       DocValuesType dvType = DocValuesType.valueOf(stripPrefix(TYPE));
-      assert dvType != null;
+      assert dvType != DocValuesType.NO;
       if (dvType == DocValuesType.NUMERIC) {
         readLine();
         assert startsWith(MINVALUE): "got " + scratch.get().utf8ToString() + " field=" + fieldName + " ext=" + ext;

Modified: lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java Fri Oct 31 16:17:25 2014
@@ -30,7 +30,7 @@ import org.apache.lucene.codecs.DocValue
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.index.FieldInfo.DocValuesType;
+import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
@@ -71,8 +71,8 @@ class SimpleTextDocValuesWriter extends 
   @Override
   public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
     assert fieldSeen(field.name);
-    assert field.getDocValuesType() == FieldInfo.DocValuesType.NUMERIC || field.hasNorms();
-    writeFieldEntry(field, FieldInfo.DocValuesType.NUMERIC);
+    assert field.getDocValuesType() == DocValuesType.NUMERIC || field.hasNorms();
+    writeFieldEntry(field, DocValuesType.NUMERIC);
 
     // first pass to find min/max
     long minValue = Long.MAX_VALUE;
@@ -145,7 +145,7 @@ class SimpleTextDocValuesWriter extends 
       final int length = value == null ? 0 : value.length;
       maxLength = Math.max(maxLength, length);
     }
-    writeFieldEntry(field, FieldInfo.DocValuesType.BINARY);
+    writeFieldEntry(field, DocValuesType.BINARY);
 
     // write maxLength
     SimpleTextUtil.write(data, MAXLENGTH);
@@ -198,7 +198,7 @@ class SimpleTextDocValuesWriter extends 
   public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
     assert fieldSeen(field.name);
     assert field.getDocValuesType() == DocValuesType.SORTED;
-    writeFieldEntry(field, FieldInfo.DocValuesType.SORTED);
+    writeFieldEntry(field, DocValuesType.SORTED);
 
     int valueCount = 0;
     int maxLength = -1;
@@ -317,7 +317,7 @@ class SimpleTextDocValuesWriter extends 
   public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws IOException {
     assert fieldSeen(field.name);
     assert field.getDocValuesType() == DocValuesType.SORTED_SET;
-    writeFieldEntry(field, FieldInfo.DocValuesType.SORTED_SET);
+    writeFieldEntry(field, DocValuesType.SORTED_SET);
 
     long valueCount = 0;
     int maxLength = 0;
@@ -423,7 +423,7 @@ class SimpleTextDocValuesWriter extends 
   }
 
   /** write the header for this field */
-  private void writeFieldEntry(FieldInfo field, FieldInfo.DocValuesType type) throws IOException {
+  private void writeFieldEntry(FieldInfo field, DocValuesType type) throws IOException {
     SimpleTextUtil.write(data, FIELD);
     SimpleTextUtil.write(data, field.name, scratch);
     SimpleTextUtil.writeNewline(data);

Modified: lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosFormat.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosFormat.java Fri Oct 31 16:17:25 2014
@@ -24,12 +24,12 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.lucene.codecs.FieldInfosFormat;
+import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.SegmentInfo;
-import org.apache.lucene.index.FieldInfo.DocValuesType;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
@@ -88,15 +88,10 @@ public class SimpleTextFieldInfosFormat 
         assert StringHelper.startsWith(scratch.get(), NUMBER);
         int fieldNumber = Integer.parseInt(readString(NUMBER.length, scratch));
 
-        final IndexOptions indexOptions;
         SimpleTextUtil.readLine(input, scratch);
         assert StringHelper.startsWith(scratch.get(), INDEXOPTIONS);
         String s = readString(INDEXOPTIONS.length, scratch);
-        if ("null".equals(s)) {
-          indexOptions = null;
-        } else {
-          indexOptions = IndexOptions.valueOf(s);
-        }
+        final IndexOptions indexOptions = IndexOptions.valueOf(s);
 
         SimpleTextUtil.readLine(input, scratch);
         assert StringHelper.startsWith(scratch.get(), STORETV);
@@ -154,11 +149,7 @@ public class SimpleTextFieldInfosFormat 
   }
 
   public DocValuesType docValuesType(String dvType) {
-    if ("false".equals(dvType)) {
-      return null;
-    } else {
-      return DocValuesType.valueOf(dvType);
-    }
+    return DocValuesType.valueOf(dvType);
   }
   
   private String readString(int offset, BytesRefBuilder scratch) {
@@ -187,12 +178,8 @@ public class SimpleTextFieldInfosFormat 
         
         SimpleTextUtil.write(out, INDEXOPTIONS);
         IndexOptions indexOptions = fi.getIndexOptions();
-        if (indexOptions != null) {
-          assert fi.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 || !fi.hasPayloads();
-          SimpleTextUtil.write(out, fi.getIndexOptions().toString(), scratch);
-        } else {
-          SimpleTextUtil.write(out, "null", scratch);
-        }
+        assert indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 || !fi.hasPayloads();
+        SimpleTextUtil.write(out, indexOptions.toString(), scratch);
         SimpleTextUtil.writeNewline(out);
         
         SimpleTextUtil.write(out, STORETV);
@@ -245,6 +232,6 @@ public class SimpleTextFieldInfosFormat 
   }
   
   private static String getDocValuesType(DocValuesType type) {
-    return type == null ? "false" : type.toString();
+    return type.toString();
   }
 }

Modified: lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java (original)
+++ lucene/dev/branches/branch_5x/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java Fri Oct 31 16:17:25 2014
@@ -17,16 +17,6 @@ package org.apache.lucene.codecs.simplet
  * limitations under the License.
  */
 
-import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.DOC;
-import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.END;
-import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.END_OFFSET;
-import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.FIELD;
-import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.FREQ;
-import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.PAYLOAD;
-import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.POS;
-import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.START_OFFSET;
-import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.TERM;
-
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 import java.util.Collections;
@@ -39,8 +29,8 @@ import org.apache.lucene.codecs.FieldsPr
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
@@ -57,11 +47,9 @@ import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.StringHelper;
-import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util.fst.Builder;
 import org.apache.lucene.util.fst.BytesRefFSTEnum;
 import org.apache.lucene.util.fst.FST;
@@ -69,6 +57,16 @@ import org.apache.lucene.util.fst.PairOu
 import org.apache.lucene.util.fst.PositiveIntOutputs;
 import org.apache.lucene.util.fst.Util;
 
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.DOC;
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.END;
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.END_OFFSET;
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.FIELD;
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.FREQ;
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.PAYLOAD;
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.POS;
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.START_OFFSET;
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.TERM;
+
 class SimpleTextFieldsReader extends FieldsProducer {
 
   private static final long BASE_RAM_BYTES_USED =
@@ -205,7 +203,7 @@ class SimpleTextFieldsReader extends Fie
 
     @Override
     public long totalTermFreq() {
-      return indexOptions == IndexOptions.DOCS_ONLY ? -1 : totalTermFreq;
+      return indexOptions == IndexOptions.DOCS ? -1 : totalTermFreq;
     }
  
     @Override
@@ -216,7 +214,7 @@ class SimpleTextFieldsReader extends Fie
       } else {
         docsEnum = new SimpleTextDocsEnum();
       }
-      return docsEnum.reset(docsStart, liveDocs, indexOptions == IndexOptions.DOCS_ONLY, docFreq);
+      return docsEnum.reset(docsStart, liveDocs, indexOptions == IndexOptions.DOCS, docFreq);
     }
 
     @Override
@@ -626,7 +624,7 @@ class SimpleTextFieldsReader extends Fie
 
     @Override
     public long getSumTotalTermFreq() {
-      return fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY ? -1 : sumTotalTermFreq;
+      return fieldInfo.getIndexOptions() == IndexOptions.DOCS ? -1 : sumTotalTermFreq;
     }
 
     @Override

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java Fri Oct 31 16:17:25 2014
@@ -28,11 +28,11 @@ import org.apache.lucene.index.BinaryDoc
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FilteredTermsEnum;
 import org.apache.lucene.index.MergeState;
-import org.apache.lucene.index.FieldInfo.DocValuesType;
+import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.MultiDocValues.OrdinalMap;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.NumericDocValues;
-import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.SegmentWriteState; // javadocs
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
@@ -138,7 +138,7 @@ public abstract class DocValuesConsumer 
 
     for (FieldInfo mergeFieldInfo : mergeState.mergeFieldInfos) {
       DocValuesType type = mergeFieldInfo.getDocValuesType();
-      if (type != null) {
+      if (type != DocValuesType.NO) {
         if (type == DocValuesType.NUMERIC) {
           List<NumericDocValues> toMerge = new ArrayList<>();
           List<Bits> docsWithField = new ArrayList<>();

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/PushPostingsWriterBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/PushPostingsWriterBase.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/PushPostingsWriterBase.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/PushPostingsWriterBase.java Fri Oct 31 16:17:25 2014
@@ -21,8 +21,8 @@ import java.io.IOException;
 
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java Fri Oct 31 16:17:25 2014
@@ -28,9 +28,9 @@ import org.apache.lucene.codecs.CodecUti
 import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.index.CorruptIndexException;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.store.IndexInput;
@@ -170,7 +170,7 @@ public final class BlockTreeTermsReader 
         if (fieldInfo == null) {
           throw new CorruptIndexException("invalid field number: " + field, termsIn);
         }
-        final long sumTotalTermFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY ? -1 : termsIn.readVLong();
+        final long sumTotalTermFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS ? -1 : termsIn.readVLong();
         final long sumDocFreq = termsIn.readVLong();
         final int docCount = termsIn.readVInt();
         final int longsSize = termsIn.readVInt();

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java Fri Oct 31 16:17:25 2014
@@ -25,11 +25,11 @@ import org.apache.lucene.codecs.BlockTer
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.PostingsWriterBase;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
@@ -695,7 +695,7 @@ public final class BlockTreeTermsWriter 
 
           // Write term stats, to separate byte[] blob:
           statsWriter.writeVInt(state.docFreq);
-          if (fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
+          if (fieldInfo.getIndexOptions() != IndexOptions.DOCS) {
             assert state.totalTermFreq >= state.docFreq: state.totalTermFreq + " vs " + state.docFreq;
             statsWriter.writeVLong(state.totalTermFreq - state.docFreq);
           }
@@ -736,7 +736,7 @@ public final class BlockTreeTermsWriter 
 
             // Write term stats, to separate byte[] blob:
             statsWriter.writeVInt(state.docFreq);
-            if (fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
+            if (fieldInfo.getIndexOptions() != IndexOptions.DOCS) {
               assert state.totalTermFreq >= state.docFreq;
               statsWriter.writeVLong(state.totalTermFreq - state.docFreq);
             }
@@ -824,6 +824,7 @@ public final class BlockTreeTermsWriter 
 
     TermsWriter(FieldInfo fieldInfo) {
       this.fieldInfo = fieldInfo;
+      assert fieldInfo.getIndexOptions() != IndexOptions.NO;
       docsSeen = new FixedBitSet(maxDoc);
 
       this.longsSize = postingsWriter.setField(fieldInfo);
@@ -843,7 +844,7 @@ public final class BlockTreeTermsWriter 
       BlockTermState state = postingsWriter.writeTerm(text, termsEnum, docsSeen);
       if (state != null) {
         assert state.docFreq != 0;
-        assert fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY || state.totalTermFreq >= state.docFreq: "postingsWriter=" + postingsWriter;
+        assert fieldInfo.getIndexOptions() == IndexOptions.DOCS || state.totalTermFreq >= state.docFreq: "postingsWriter=" + postingsWriter;
         sumDocFreq += state.docFreq;
         sumTotalTermFreq += state.totalTermFreq;
         pushTerm(text);
@@ -944,7 +945,7 @@ public final class BlockTreeTermsWriter 
                                      longsSize,
                                      minTerm, maxTerm));
       } else {
-        assert sumTotalTermFreq == 0 || fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY && sumTotalTermFreq == -1;
+        assert sumTotalTermFreq == 0 || fieldInfo.getIndexOptions() == IndexOptions.DOCS && sumTotalTermFreq == -1;
         assert sumDocFreq == 0;
         assert docsSeen.cardinality() == 0;
       }
@@ -974,7 +975,8 @@ public final class BlockTreeTermsWriter 
         termsOut.writeVLong(field.numTerms);
         termsOut.writeVInt(field.rootCode.length);
         termsOut.writeBytes(field.rootCode.bytes, field.rootCode.offset, field.rootCode.length);
-        if (field.fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
+        assert field.fieldInfo.getIndexOptions() != IndexOptions.NO;
+        if (field.fieldInfo.getIndexOptions() != IndexOptions.DOCS) {
           termsOut.writeVLong(field.sumTotalTermFreq);
         }
         termsOut.writeVLong(field.sumDocFreq);

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java Fri Oct 31 16:17:25 2014
@@ -21,7 +21,7 @@ import java.io.IOException;
 import java.util.Collections;
 
 import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.ByteArrayDataInput;

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java Fri Oct 31 16:17:25 2014
@@ -21,7 +21,7 @@ import java.io.IOException;
 
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.IndexInput;

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java Fri Oct 31 16:17:25 2014
@@ -20,7 +20,7 @@ package org.apache.lucene.codecs.blocktr
 import java.io.IOException;
 
 import org.apache.lucene.codecs.BlockTermState;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
@@ -278,7 +278,7 @@ final class IntersectTermsEnumFrame {
       // stats
       termState.docFreq = statsReader.readVInt();
       //if (DEBUG) System.out.println("    dF=" + state.docFreq);
-      if (ite.fr.fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
+      if (ite.fr.fieldInfo.getIndexOptions() != IndexOptions.DOCS) {
         termState.totalTermFreq = termState.docFreq + statsReader.readVLong();
         //if (DEBUG) System.out.println("    totTF=" + state.totalTermFreq);
       }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java Fri Oct 31 16:17:25 2014
@@ -23,7 +23,7 @@ import java.io.PrintStream;
 import org.apache.lucene.codecs.BlockTermState;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.ByteArrayDataInput;

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java Fri Oct 31 16:17:25 2014
@@ -20,7 +20,7 @@ package org.apache.lucene.codecs.blocktr
 import java.io.IOException;
 
 import org.apache.lucene.codecs.BlockTermState;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.TermsEnum.SeekStatus;
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.util.ArrayUtil;
@@ -403,7 +403,7 @@ final class SegmentTermsEnumFrame {
       // stats
       state.docFreq = statsReader.readVInt();
       //if (DEBUG) System.out.println("    dF=" + state.docFreq);
-      if (ste.fr.fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
+      if (ste.fr.fieldInfo.getIndexOptions() != IndexOptions.DOCS) {
         state.totalTermFreq = state.docFreq + statsReader.readVLong();
         //if (DEBUG) System.out.println("    totTF=" + state.totalTermFreq);
       }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesFormat.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesFormat.java Fri Oct 31 16:17:25 2014
@@ -21,11 +21,11 @@ import java.io.IOException;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.DocValuesConsumer;
-import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.util.SmallFloat;
 import org.apache.lucene.util.fst.FST;

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java Fri Oct 31 16:17:25 2014
@@ -25,12 +25,12 @@ import org.apache.lucene.codecs.CodecUti
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FieldInfosFormat;
 import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.SegmentInfo;
-import org.apache.lucene.index.FieldInfo.DocValuesType;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.Directory;
@@ -159,35 +159,33 @@ public final class Lucene50FieldInfosFor
   static {
     // We "mirror" DocValues enum values with the constants below; let's try to ensure if we add a new DocValuesType while this format is
     // still used for writing, we remember to fix this encoding:
-    assert DocValuesType.values().length == 5;
+    assert DocValuesType.values().length == 6;
   }
 
   private static byte docValuesByte(DocValuesType type) {
-    if (type == null) {
+    switch(type) {
+    case NO:
       return 0;
-    } else {
-      switch(type) {
-      case NUMERIC:
-        return 1;
-      case BINARY:
-        return 2;
-      case SORTED:
-        return 3;
-      case SORTED_SET:
-        return 4;
-      case SORTED_NUMERIC:
-        return 5;
-      default:
-        // BUG
-        throw new AssertionError("unhandled DocValuesType: " + type);
-      }
+    case NUMERIC:
+      return 1;
+    case BINARY:
+      return 2;
+    case SORTED:
+      return 3;
+    case SORTED_SET:
+      return 4;
+    case SORTED_NUMERIC:
+      return 5;
+    default:
+      // BUG
+      throw new AssertionError("unhandled DocValuesType: " + type);
     }
   }
 
   private static DocValuesType getDocValuesType(IndexInput input, byte b) throws IOException {
     switch(b) {
     case 0:
-      return null;
+      return DocValuesType.NO;
     case 1:
       return DocValuesType.NUMERIC;
     case 2:
@@ -206,35 +204,33 @@ public final class Lucene50FieldInfosFor
   static {
     // We "mirror" IndexOptions enum values with the constants below; let's try to ensure if we add a new IndexOption while this format is
     // still used for writing, we remember to fix this encoding:
-    assert IndexOptions.values().length == 4;
+    assert IndexOptions.values().length == 5;
   }
 
   private static byte indexOptionsByte(IndexOptions indexOptions) {
-    if (indexOptions == null) {
+    switch (indexOptions) {
+    case NO:
       return 0;
-    } else {
-      switch (indexOptions) {
-      case DOCS_ONLY:
-        return 1;
-      case DOCS_AND_FREQS:
-        return 2;
-      case DOCS_AND_FREQS_AND_POSITIONS:
-        return 3;
-      case DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS:
-        return 4;
-      default:
-        // BUG:
-        throw new AssertionError("unhandled IndexOptions: " + indexOptions);
-      }
+    case DOCS:
+      return 1;
+    case DOCS_AND_FREQS:
+      return 2;
+    case DOCS_AND_FREQS_AND_POSITIONS:
+      return 3;
+    case DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS:
+      return 4;
+    default:
+      // BUG:
+      throw new AssertionError("unhandled IndexOptions: " + indexOptions);
     }
   }
   
   private static IndexOptions getIndexOptions(IndexInput input, byte b) throws IOException {
     switch (b) {
     case 0:
-      return null;
+      return IndexOptions.NO;
     case 1:
-      return IndexOptions.DOCS_ONLY;
+      return IndexOptions.DOCS;
     case 2:
       return IndexOptions.DOCS_AND_FREQS;
     case 3:

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java Fri Oct 31 16:17:25 2014
@@ -31,7 +31,7 @@ import org.apache.lucene.codecs.Postings
 import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader;
 import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
 import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.TermState;
@@ -186,7 +186,7 @@ import org.apache.lucene.util.packed.Pac
  *
  * <p>The .doc file contains the lists of documents which contain each term, along
  * with the frequency of the term in that document (except when frequencies are
- * omitted: {@link IndexOptions#DOCS_ONLY}). It also saves skip data to the beginning of 
+ * omitted: {@link IndexOptions#DOCS}). It also saves skip data to the beginning of 
  * each packed or VInt block, when the length of document list is larger than packed block size.</p>
  *
  * <ul>
@@ -232,7 +232,7 @@ import org.apache.lucene.util.packed.Pac
  *          and three times in document eleven, with frequencies indexed, would be the
  *          following sequence of VInts:</p>
  *       <p>15, 8, 3</p>
- *       <p>If frequencies were omitted ({@link IndexOptions#DOCS_ONLY}) it would be this
+ *       <p>If frequencies were omitted ({@link IndexOptions#DOCS}) it would be this
  *          sequence of VInts instead:</p>
  *       <p>7,4</p>
  *   </li>

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java Fri Oct 31 16:17:25 2014
@@ -17,17 +17,6 @@ package org.apache.lucene.codecs.lucene5
  * limitations under the License.
  */
 
-import static org.apache.lucene.codecs.lucene50.ForUtil.MAX_DATA_SIZE;
-import static org.apache.lucene.codecs.lucene50.ForUtil.MAX_ENCODED_SIZE;
-import static org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.BLOCK_SIZE;
-import static org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.DOC_CODEC;
-import static org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.MAX_SKIP_LEVELS;
-import static org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.PAY_CODEC;
-import static org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.POS_CODEC;
-import static org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.TERMS_CODEC;
-import static org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.VERSION_CURRENT;
-import static org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.VERSION_START;
-
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collections;
@@ -39,8 +28,8 @@ import org.apache.lucene.codecs.lucene50
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.IndexInput;
@@ -51,6 +40,17 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.RamUsageEstimator;
 
+import static org.apache.lucene.codecs.lucene50.ForUtil.MAX_DATA_SIZE;
+import static org.apache.lucene.codecs.lucene50.ForUtil.MAX_ENCODED_SIZE;
+import static org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.BLOCK_SIZE;
+import static org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.DOC_CODEC;
+import static org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.MAX_SKIP_LEVELS;
+import static org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.PAY_CODEC;
+import static org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.POS_CODEC;
+import static org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.TERMS_CODEC;
+import static org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.VERSION_CURRENT;
+import static org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.VERSION_START;
+
 /**
  * Concrete class that reads docId(maybe frq,pos,offset,payloads) list
  * with postings format.

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/BinaryDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/BinaryDocValuesField.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/BinaryDocValuesField.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/BinaryDocValuesField.java Fri Oct 31 16:17:25 2014
@@ -18,7 +18,7 @@ package org.apache.lucene.document;
  */
 
 import org.apache.lucene.index.BinaryDocValues;
-import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.util.BytesRef;
 
 /**
@@ -46,7 +46,7 @@ public class BinaryDocValuesField extend
    */
   public static final FieldType TYPE = new FieldType();
   static {
-    TYPE.setDocValueType(FieldInfo.DocValuesType.BINARY);
+    TYPE.setDocValueType(DocValuesType.BINARY);
     TYPE.freeze();
   }
   

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/DoubleField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/DoubleField.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/DoubleField.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/DoubleField.java Fri Oct 31 16:17:25 2014
@@ -18,7 +18,7 @@ package org.apache.lucene.document;
  */
 
 import org.apache.lucene.analysis.NumericTokenStream; // javadocs
-import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.search.NumericRangeFilter; // javadocs
 import org.apache.lucene.search.NumericRangeQuery; // javadocs
 import org.apache.lucene.util.NumericUtils;
@@ -121,7 +121,7 @@ public final class DoubleField extends F
   static {
     TYPE_NOT_STORED.setTokenized(true);
     TYPE_NOT_STORED.setOmitNorms(true);
-    TYPE_NOT_STORED.setIndexOptions(IndexOptions.DOCS_ONLY);
+    TYPE_NOT_STORED.setIndexOptions(IndexOptions.DOCS);
     TYPE_NOT_STORED.setNumericType(FieldType.NumericType.DOUBLE);
     TYPE_NOT_STORED.freeze();
   }
@@ -134,7 +134,7 @@ public final class DoubleField extends F
   static {
     TYPE_STORED.setTokenized(true);
     TYPE_STORED.setOmitNorms(true);
-    TYPE_STORED.setIndexOptions(IndexOptions.DOCS_ONLY);
+    TYPE_STORED.setIndexOptions(IndexOptions.DOCS);
     TYPE_STORED.setNumericType(FieldType.NumericType.DOUBLE);
     TYPE_STORED.setStored(true);
     TYPE_STORED.freeze();

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/Field.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/Field.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/Field.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/Field.java Fri Oct 31 16:17:25 2014
@@ -26,8 +26,8 @@ import org.apache.lucene.analysis.TokenS
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.document.FieldType.NumericType;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.FieldInvertState; // javadocs
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.IndexWriter; // javadocs
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.IndexableFieldType;
@@ -121,7 +121,7 @@ public class Field implements IndexableF
     if (type.stored()) {
       throw new IllegalArgumentException("fields with a Reader value cannot be stored");
     }
-    if (type.indexOptions() != null && !type.tokenized()) {
+    if (type.indexOptions() != IndexOptions.NO && !type.tokenized()) {
       throw new IllegalArgumentException("non-tokenized fields must use String values");
     }
     
@@ -147,7 +147,7 @@ public class Field implements IndexableF
     if (tokenStream == null) {
       throw new NullPointerException("tokenStream cannot be null");
     }
-    if (type.indexOptions() == null || !type.tokenized()) {
+    if (type.indexOptions() == IndexOptions.NO || !type.tokenized()) {
       throw new IllegalArgumentException("TokenStream fields must be indexed and tokenized");
     }
     if (type.stored()) {
@@ -213,7 +213,7 @@ public class Field implements IndexableF
     if (bytes == null) {
       throw new IllegalArgumentException("bytes cannot be null");
     }
-    if (type.indexOptions() != null) {
+    if (type.indexOptions() != IndexOptions.NO) {
       throw new IllegalArgumentException("Fields with BytesRef values cannot be indexed");
     }
     this.fieldsData = bytes;
@@ -240,7 +240,7 @@ public class Field implements IndexableF
     if (value == null) {
       throw new IllegalArgumentException("value cannot be null");
     }
-    if (!type.stored() && type.indexOptions() == null) {
+    if (!type.stored() && type.indexOptions() == IndexOptions.NO) {
       throw new IllegalArgumentException("it doesn't make sense to have a field that "
         + "is neither indexed nor stored");
     }
@@ -337,7 +337,7 @@ public class Field implements IndexableF
     if (!(fieldsData instanceof BytesRef)) {
       throw new IllegalArgumentException("cannot change value type from " + fieldsData.getClass().getSimpleName() + " to BytesRef");
     }
-    if (type.indexOptions() != null) {
+    if (type.indexOptions() != IndexOptions.NO) {
       throw new IllegalArgumentException("cannot set a BytesRef value on an indexed field");
     }
     if (value == null) {
@@ -418,7 +418,7 @@ public class Field implements IndexableF
    * values from stringValue() or getBinaryValue()
    */
   public void setTokenStream(TokenStream tokenStream) {
-    if (type.indexOptions() == null || !type.tokenized()) {
+    if (type.indexOptions() == IndexOptions.NO || !type.tokenized()) {
       throw new IllegalArgumentException("TokenStream fields must be indexed and tokenized");
     }
     if (type.numericType() != null) {
@@ -451,7 +451,7 @@ public class Field implements IndexableF
    */
   public void setBoost(float boost) {
     if (boost != 1.0f) {
-      if (type.indexOptions() == null || type.omitNorms()) {
+      if (type.indexOptions() == IndexOptions.NO || type.omitNorms()) {
         throw new IllegalArgumentException("You cannot set an index-time boost on an unindexed field, or one that omits norms");
       }
     }
@@ -501,7 +501,7 @@ public class Field implements IndexableF
 
   @Override
   public TokenStream tokenStream(Analyzer analyzer, TokenStream reuse) throws IOException {
-    if (fieldType().indexOptions() == null) {
+    if (fieldType().indexOptions() == IndexOptions.NO) {
       // Not indexed
       return null;
     }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/FieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/FieldType.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/FieldType.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/FieldType.java Fri Oct 31 16:17:25 2014
@@ -18,8 +18,8 @@ package org.apache.lucene.document;
  */
 
 import org.apache.lucene.analysis.Analyzer; // javadocs
-import org.apache.lucene.index.FieldInfo.DocValuesType;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.DocValuesType;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.IndexableFieldType;
 import org.apache.lucene.search.NumericRangeQuery; // javadocs
 import org.apache.lucene.util.NumericUtils;
@@ -50,11 +50,11 @@ public class FieldType implements Indexa
   private boolean storeTermVectorPositions;
   private boolean storeTermVectorPayloads;
   private boolean omitNorms;
-  private IndexOptions indexOptions;
+  private IndexOptions indexOptions = IndexOptions.NO;
   private NumericType numericType;
   private boolean frozen;
   private int numericPrecisionStep = NumericUtils.PRECISION_STEP_DEFAULT;
-  private DocValuesType docValueType;
+  private DocValuesType docValueType = DocValuesType.NO;
 
   /**
    * Create a new mutable FieldType with all of the properties from <code>ref</code>
@@ -263,7 +263,7 @@ public class FieldType implements Indexa
    * {@inheritDoc}
    * <p>
    * The default is {@link IndexOptions#DOCS_AND_FREQS_AND_POSITIONS}.
-   * @see #setIndexOptions(org.apache.lucene.index.FieldInfo.IndexOptions)
+   * @see #setIndexOptions(IndexOptions)
    */
   @Override
   public IndexOptions indexOptions() {
@@ -279,6 +279,9 @@ public class FieldType implements Indexa
    */
   public void setIndexOptions(IndexOptions value) {
     checkIfFrozen();
+    if (value == null) {
+      throw new NullPointerException("IndexOptions cannot be null");
+    }
     this.indexOptions = value;
   }
 
@@ -341,7 +344,7 @@ public class FieldType implements Indexa
     if (stored()) {
       result.append("stored");
     }
-    if (indexOptions != null) {
+    if (indexOptions != IndexOptions.NO) {
       if (result.length() > 0)
         result.append(",");
       result.append("indexed");
@@ -374,9 +377,10 @@ public class FieldType implements Indexa
         result.append(numericPrecisionStep);
       }
     }
-    if (docValueType != null) {
-      if (result.length() > 0)
+    if (docValueType != DocValuesType.NO) {
+      if (result.length() > 0) {
         result.append(",");
+      }
       result.append("docValueType=");
       result.append(docValueType);
     }
@@ -388,7 +392,7 @@ public class FieldType implements Indexa
    * {@inheritDoc}
    * <p>
    * The default is <code>null</code> (no docValues) 
-   * @see #setDocValueType(org.apache.lucene.index.FieldInfo.DocValuesType)
+   * @see #setDocValueType(DocValuesType)
    */
   @Override
   public DocValuesType docValueType() {
@@ -404,6 +408,9 @@ public class FieldType implements Indexa
    */
   public void setDocValueType(DocValuesType type) {
     checkIfFrozen();
+    if (type == null) {
+      throw new NullPointerException("DocValuesType cannot be null");
+    }
     docValueType = type;
   }
 
@@ -412,7 +419,7 @@ public class FieldType implements Indexa
     final int prime = 31;
     int result = 1;
     result = prime * result + ((docValueType == null) ? 0 : docValueType.hashCode());
-    result = prime * result + ((indexOptions == null) ? 0 : indexOptions.hashCode());
+    result = prime * result + indexOptions.hashCode();
     result = prime * result + numericPrecisionStep;
     result = prime * result + ((numericType == null) ? 0 : numericType.hashCode());
     result = prime * result + (omitNorms ? 1231 : 1237);

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/FloatField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/FloatField.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/FloatField.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/FloatField.java Fri Oct 31 16:17:25 2014
@@ -18,7 +18,7 @@ package org.apache.lucene.document;
  */
 
 import org.apache.lucene.analysis.NumericTokenStream; // javadocs
-import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.search.NumericRangeFilter; // javadocs
 import org.apache.lucene.search.NumericRangeQuery; // javadocs
 import org.apache.lucene.util.NumericUtils;
@@ -121,7 +121,7 @@ public final class FloatField extends Fi
   static {
     TYPE_NOT_STORED.setTokenized(true);
     TYPE_NOT_STORED.setOmitNorms(true);
-    TYPE_NOT_STORED.setIndexOptions(IndexOptions.DOCS_ONLY);
+    TYPE_NOT_STORED.setIndexOptions(IndexOptions.DOCS);
     TYPE_NOT_STORED.setNumericType(FieldType.NumericType.FLOAT);
     TYPE_NOT_STORED.setNumericPrecisionStep(NumericUtils.PRECISION_STEP_DEFAULT_32);
     TYPE_NOT_STORED.freeze();
@@ -135,7 +135,7 @@ public final class FloatField extends Fi
   static {
     TYPE_STORED.setTokenized(true);
     TYPE_STORED.setOmitNorms(true);
-    TYPE_STORED.setIndexOptions(IndexOptions.DOCS_ONLY);
+    TYPE_STORED.setIndexOptions(IndexOptions.DOCS);
     TYPE_STORED.setNumericType(FieldType.NumericType.FLOAT);
     TYPE_STORED.setNumericPrecisionStep(NumericUtils.PRECISION_STEP_DEFAULT_32);
     TYPE_STORED.setStored(true);

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/IntField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/IntField.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/IntField.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/IntField.java Fri Oct 31 16:17:25 2014
@@ -18,7 +18,7 @@ package org.apache.lucene.document;
  */
 
 import org.apache.lucene.analysis.NumericTokenStream; // javadocs
-import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.search.NumericRangeFilter; // javadocs
 import org.apache.lucene.search.NumericRangeQuery; // javadocs
 import org.apache.lucene.util.NumericUtils;
@@ -121,7 +121,7 @@ public final class IntField extends Fiel
   static {
     TYPE_NOT_STORED.setTokenized(true);
     TYPE_NOT_STORED.setOmitNorms(true);
-    TYPE_NOT_STORED.setIndexOptions(IndexOptions.DOCS_ONLY);
+    TYPE_NOT_STORED.setIndexOptions(IndexOptions.DOCS);
     TYPE_NOT_STORED.setNumericType(FieldType.NumericType.INT);
     TYPE_NOT_STORED.setNumericPrecisionStep(NumericUtils.PRECISION_STEP_DEFAULT_32);
     TYPE_NOT_STORED.freeze();
@@ -135,7 +135,7 @@ public final class IntField extends Fiel
   static {
     TYPE_STORED.setTokenized(true);
     TYPE_STORED.setOmitNorms(true);
-    TYPE_STORED.setIndexOptions(IndexOptions.DOCS_ONLY);
+    TYPE_STORED.setIndexOptions(IndexOptions.DOCS);
     TYPE_STORED.setNumericType(FieldType.NumericType.INT);
     TYPE_STORED.setNumericPrecisionStep(NumericUtils.PRECISION_STEP_DEFAULT_32);
     TYPE_STORED.setStored(true);

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/LongField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/LongField.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/LongField.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/LongField.java Fri Oct 31 16:17:25 2014
@@ -18,7 +18,7 @@ package org.apache.lucene.document;
  */
 
 import org.apache.lucene.analysis.NumericTokenStream; // javadocs
-import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.search.NumericRangeFilter; // javadocs
 import org.apache.lucene.search.NumericRangeQuery; // javadocs
 import org.apache.lucene.util.NumericUtils;
@@ -131,7 +131,7 @@ public final class LongField extends Fie
   static {
     TYPE_NOT_STORED.setTokenized(true);
     TYPE_NOT_STORED.setOmitNorms(true);
-    TYPE_NOT_STORED.setIndexOptions(IndexOptions.DOCS_ONLY);
+    TYPE_NOT_STORED.setIndexOptions(IndexOptions.DOCS);
     TYPE_NOT_STORED.setNumericType(FieldType.NumericType.LONG);
     TYPE_NOT_STORED.freeze();
   }
@@ -144,7 +144,7 @@ public final class LongField extends Fie
   static {
     TYPE_STORED.setTokenized(true);
     TYPE_STORED.setOmitNorms(true);
-    TYPE_STORED.setIndexOptions(IndexOptions.DOCS_ONLY);
+    TYPE_STORED.setIndexOptions(IndexOptions.DOCS);
     TYPE_STORED.setNumericType(FieldType.NumericType.LONG);
     TYPE_STORED.setStored(true);
     TYPE_STORED.freeze();

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/NumericDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/NumericDocValuesField.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/NumericDocValuesField.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/NumericDocValuesField.java Fri Oct 31 16:17:25 2014
@@ -17,7 +17,7 @@ package org.apache.lucene.document;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.DocValuesType;
 
 /**
  * <p>
@@ -40,7 +40,7 @@ public class NumericDocValuesField exten
    */
   public static final FieldType TYPE = new FieldType();
   static {
-    TYPE.setDocValueType(FieldInfo.DocValuesType.NUMERIC);
+    TYPE.setDocValueType(DocValuesType.NUMERIC);
     TYPE.freeze();
   }
 

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/SortedDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/SortedDocValuesField.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/SortedDocValuesField.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/SortedDocValuesField.java Fri Oct 31 16:17:25 2014
@@ -17,7 +17,7 @@ package org.apache.lucene.document;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.util.BytesRef;
 
 /**
@@ -43,7 +43,7 @@ public class SortedDocValuesField extend
    */
   public static final FieldType TYPE = new FieldType();
   static {
-    TYPE.setDocValueType(FieldInfo.DocValuesType.SORTED);
+    TYPE.setDocValueType(DocValuesType.SORTED);
     TYPE.freeze();
   }
 

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/SortedNumericDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/SortedNumericDocValuesField.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/SortedNumericDocValuesField.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/SortedNumericDocValuesField.java Fri Oct 31 16:17:25 2014
@@ -17,7 +17,7 @@ package org.apache.lucene.document;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.util.NumericUtils;
 
 /**
@@ -50,7 +50,7 @@ public class SortedNumericDocValuesField
    */
   public static final FieldType TYPE = new FieldType();
   static {
-    TYPE.setDocValueType(FieldInfo.DocValuesType.SORTED_NUMERIC);
+    TYPE.setDocValueType(DocValuesType.SORTED_NUMERIC);
     TYPE.freeze();
   }
 

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/SortedSetDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/SortedSetDocValuesField.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/SortedSetDocValuesField.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/SortedSetDocValuesField.java Fri Oct 31 16:17:25 2014
@@ -17,7 +17,7 @@ package org.apache.lucene.document;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.util.BytesRef;
 
 /**
@@ -44,7 +44,7 @@ public class SortedSetDocValuesField ext
    */
   public static final FieldType TYPE = new FieldType();
   static {
-    TYPE.setDocValueType(FieldInfo.DocValuesType.SORTED_SET);
+    TYPE.setDocValueType(DocValuesType.SORTED_SET);
     TYPE.freeze();
   }
 

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/StringField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/StringField.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/StringField.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/StringField.java Fri Oct 31 16:17:25 2014
@@ -17,7 +17,7 @@ package org.apache.lucene.document;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.IndexOptions;
 
 /** A field that is indexed but not tokenized: the entire
  *  String value is indexed as a single token.  For example
@@ -37,12 +37,12 @@ public final class StringField extends F
 
   static {
     TYPE_NOT_STORED.setOmitNorms(true);
-    TYPE_NOT_STORED.setIndexOptions(IndexOptions.DOCS_ONLY);
+    TYPE_NOT_STORED.setIndexOptions(IndexOptions.DOCS);
     TYPE_NOT_STORED.setTokenized(false);
     TYPE_NOT_STORED.freeze();
 
     TYPE_STORED.setOmitNorms(true);
-    TYPE_STORED.setIndexOptions(IndexOptions.DOCS_ONLY);
+    TYPE_STORED.setIndexOptions(IndexOptions.DOCS);
     TYPE_STORED.setStored(true);
     TYPE_STORED.setTokenized(false);
     TYPE_STORED.freeze();

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/TextField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/TextField.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/TextField.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/document/TextField.java Fri Oct 31 16:17:25 2014
@@ -20,7 +20,7 @@ package org.apache.lucene.document;
 import java.io.Reader;
 
 import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.IndexOptions;
 
 /** A field that is indexed and tokenized, without term
  *  vectors.  For example this would be used on a 'body'

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java Fri Oct 31 16:17:25 2014
@@ -1,15 +1,5 @@
 package org.apache.lucene.index;
 
-import org.apache.lucene.document.BinaryDocValuesField;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.InPlaceMergeSorter;
-import org.apache.lucene.util.packed.PackedInts;
-import org.apache.lucene.util.packed.PagedGrowableWriter;
-import org.apache.lucene.util.packed.PagedMutable;
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -27,6 +17,15 @@ import org.apache.lucene.util.packed.Pag
  * limitations under the License.
  */
 
+import org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.InPlaceMergeSorter;
+import org.apache.lucene.util.packed.PackedInts;
+import org.apache.lucene.util.packed.PagedGrowableWriter;
+import org.apache.lucene.util.packed.PagedMutable;
+
 /**
  * A {@link DocValuesFieldUpdates} which holds updates of documents, of a single
  * {@link BinaryDocValuesField}.
@@ -102,7 +101,7 @@ class BinaryDocValuesFieldUpdates extend
   private final int bitsPerValue;
   
   public BinaryDocValuesFieldUpdates(String field, int maxDoc) {
-    super(field, FieldInfo.DocValuesType.BINARY);
+    super(field, DocValuesType.BINARY);
     bitsPerValue = PackedInts.bitsRequired(maxDoc - 1);
     docs = new PagedMutable(1, PAGE_SIZE, bitsPerValue, PackedInts.COMPACT);
     offsets = new PagedGrowableWriter(1, PAGE_SIZE, 1, PackedInts.FAST);

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Fri Oct 31 16:17:25 2014
@@ -34,7 +34,6 @@ import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.CheckIndex.Status.DocValuesStatus;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java Fri Oct 31 16:17:25 2014
@@ -29,8 +29,6 @@ import org.apache.lucene.codecs.NormsCon
 import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.StoredFieldsWriter;
 import org.apache.lucene.document.FieldType;
-import org.apache.lucene.index.FieldInfo.DocValuesType;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.ArrayUtil;
@@ -341,8 +339,12 @@ final class DefaultIndexingChain extends
 
     PerField fp = null;
 
+    if (fieldType.indexOptions() == null) {
+      throw new NullPointerException("IndexOptions must not be null (field: \"" + field.name() + "\")");
+    }
+
     // Invert indexed fields:
-    if (fieldType.indexOptions() != null) {
+    if (fieldType.indexOptions() != IndexOptions.NO) {
       
       // if the field omits norms, the boost cannot be indexed.
       if (fieldType.omitNorms() && field.boost() != 1.0f) {
@@ -380,7 +382,10 @@ final class DefaultIndexingChain extends
     }
 
     DocValuesType dvType = fieldType.docValueType();
-    if (dvType != null) {
+    if (dvType == null) {
+      throw new NullPointerException("docValueType cannot be null (field: \"" + fieldName + "\")");
+    }
+    if (dvType != DocValuesType.NO) {
       if (fp == null) {
         fp = getOrAddField(fieldName, fieldType, false);
       }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java?rev=1635804&r1=1635803&r2=1635804&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java Fri Oct 31 16:17:25 2014
@@ -1,12 +1,5 @@
 package org.apache.lucene.index;
 
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.lucene.index.NumericDocValuesFieldUpdates;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.util.packed.PagedGrowableWriter;
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -24,6 +17,12 @@ import org.apache.lucene.util.packed.Pag
  * limitations under the License.
  */
 
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.packed.PagedGrowableWriter;
+
 /**
  * Holds updates of a single DocValues field, for a set of documents.
  * 
@@ -98,7 +97,7 @@ abstract class DocValuesFieldUpdates {
       return ramBytesPerDoc;
     }
     
-    DocValuesFieldUpdates getUpdates(String field, FieldInfo.DocValuesType type) {
+    DocValuesFieldUpdates getUpdates(String field, DocValuesType type) {
       switch (type) {
         case NUMERIC:
           return numericDVUpdates.get(field);
@@ -109,7 +108,7 @@ abstract class DocValuesFieldUpdates {
       }
     }
     
-    DocValuesFieldUpdates newUpdates(String field, FieldInfo.DocValuesType type, int maxDoc) {
+    DocValuesFieldUpdates newUpdates(String field, DocValuesType type, int maxDoc) {
       switch (type) {
         case NUMERIC:
           assert numericDVUpdates.get(field) == null;
@@ -133,10 +132,13 @@ abstract class DocValuesFieldUpdates {
   }
   
   final String field;
-  final FieldInfo.DocValuesType type;
+  final DocValuesType type;
   
-  protected DocValuesFieldUpdates(String field, FieldInfo.DocValuesType type) {
+  protected DocValuesFieldUpdates(String field, DocValuesType type) {
     this.field = field;
+    if (type == null) {
+      throw new NullPointerException("DocValuesType cannot be null");
+    }
     this.type = type;
   }