You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2013/01/15 22:00:41 UTC

svn commit: r1433646 [1/3] - in /lucene/dev/branches/lucene4547: lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/ lucene/codecs/src/java/org/apache/lucene/codecs/sep/ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/ lucene/core/src/ja...

Author: rmuir
Date: Tue Jan 15 21:00:39 2013
New Revision: 1433646

URL: http://svn.apache.org/viewvc?rev=1433646&view=rev
Log:
progress towards full cutover to 2.0 apis

Added:
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/BinaryDocValuesField.java   (with props)
Removed:
    lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/sep/SepDocValuesConsumer.java
    lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/sep/SepDocValuesProducer.java
    lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesConsumer.java
    lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java
    lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java
    lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocConsumer.java
    lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocProducer.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/DocValuesArraySource.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/DocValuesFormat.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/NormsFormat.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/PerDocConsumer.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/PerDocProducer.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/PerDocProducerBase.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesConsumer.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesFormat.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesProducer.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsFormat.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/DocValues.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/Norm.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/PerDocWriteState.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SortedBytesMergeUtils.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/TypePromoter.java
    lucene/dev/branches/lucene4547/lucene/grouping/src/java/org/apache/lucene/search/grouping/dv/
    lucene/dev/branches/lucene4547/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/DateDocValuesFieldSource.java
    lucene/dev/branches/lucene4547/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/DocValuesFieldSource.java
    lucene/dev/branches/lucene4547/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/NumericDocValuesFieldSource.java
    lucene/dev/branches/lucene4547/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/NumericIndexDocValueSource.java
    lucene/dev/branches/lucene4547/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/StrDocValuesFieldSource.java
    lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSepDocValuesFormat.java
Modified:
    lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java
    lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java
    lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java
    lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosWriter.java
    lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSimpleDocValuesFormat.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/ByteDocValuesField.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/DerefBytesDocValuesField.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/DoubleDocValuesField.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/Field.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/FieldType.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/FloatDocValuesField.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/IntDocValuesField.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/LongDocValuesField.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/PackedLongDocValuesField.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/ShortDocValuesField.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/SortedBytesDocValuesField.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/StraightBytesDocValuesField.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/DocValuesProcessor.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/IndexableField.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/IndexableFieldType.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/NormsConsumer.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/NormsConsumerPerField.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/StorableFieldType.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/similarities/DefaultSimilarity.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/similarities/PerFieldSimilarityWrapper.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReaderReopen.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestMaxTermFrequency.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestNorms.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestOmitNorms.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestSegmentReader.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestSimpleDocValuesIndexing.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestUniqueTermCount.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/search/TestConjunctions.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/search/TestDisjunctionMaxQuery.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/search/TestDocValuesScoring.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/search/TestSimilarity.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/search/TestSimilarityProvider.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/search/TestSort.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadNearQuery.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadTermQuery.java
    lucene/dev/branches/lucene4547/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java
    lucene/dev/branches/lucene4547/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java
    lucene/dev/branches/lucene4547/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupsCollectorTest.java
    lucene/dev/branches/lucene4547/lucene/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java
    lucene/dev/branches/lucene4547/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupFacetCollectorTest.java
    lucene/dev/branches/lucene4547/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupingSearchTest.java
    lucene/dev/branches/lucene4547/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
    lucene/dev/branches/lucene4547/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
    lucene/dev/branches/lucene4547/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndexNormDocValues.java
    lucene/dev/branches/lucene4547/lucene/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java
    lucene/dev/branches/lucene4547/lucene/misc/src/java/org/apache/lucene/misc/SweetSpotSimilarity.java
    lucene/dev/branches/lucene4547/lucene/misc/src/test/org/apache/lucene/misc/SweetSpotSimilarityTest.java
    lucene/dev/branches/lucene4547/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/NormValueSource.java
    lucene/dev/branches/lucene4547/lucene/spatial/src/java/org/apache/lucene/spatial/SpatialStrategy.java
    lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java
    lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java
    lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java
    lucene/dev/branches/lucene4547/solr/core/src/test/org/apache/solr/search/TestDocSet.java
    lucene/dev/branches/lucene4547/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java
    lucene/dev/branches/lucene4547/solr/core/src/test/org/apache/solr/update/DocumentBuilderTest.java

Modified: lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java (original)
+++ lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java Tue Jan 15 21:00:39 2013
@@ -25,8 +25,8 @@ import org.apache.lucene.codecs.CodecUti
 import org.apache.lucene.codecs.SimpleDVProducer;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.CorruptIndexException;
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.NumericDocValues;
@@ -75,10 +75,10 @@ class DiskDocValuesProducer extends Simp
   private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
     int fieldNumber = meta.readVInt();
     while (fieldNumber != -1) {
-      DocValues.Type type = infos.fieldInfo(fieldNumber).getDocValuesType();
-      if (DocValues.isNumber(type) || DocValues.isFloat(type)) {
+      DocValuesType type = infos.fieldInfo(fieldNumber).getDocValuesType();
+      if (type == DocValuesType.NUMERIC) {
         numerics.put(fieldNumber, readNumericEntry(meta));
-      } else if (DocValues.isBytes(type)) {
+      } else if (type == DocValuesType.BINARY) {
         BinaryEntry b = readBinaryEntry(meta);
         binaries.put(fieldNumber, b);
         if (b.minLength != b.maxLength) {
@@ -88,7 +88,7 @@ class DiskDocValuesProducer extends Simp
           // variable length byte[]: read addresses as a numeric dv field
           numerics.put(fieldNumber, readNumericEntry(meta));
         }
-      } else if (DocValues.isSortedBytes(type)) {
+      } else if (type == DocValuesType.SORTED) {
         BinaryEntry b = readBinaryEntry(meta);
         binaries.put(fieldNumber, b);
         if (b.minLength != b.maxLength) {

Modified: lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java (original)
+++ lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java Tue Jan 15 21:00:39 2013
@@ -18,10 +18,8 @@ package org.apache.lucene.codecs.simplet
  */
 
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FieldInfosFormat;
 import org.apache.lucene.codecs.LiveDocsFormat;
-import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.SegmentInfoFormat;
 import org.apache.lucene.codecs.SimpleDocValuesFormat;
@@ -41,10 +39,6 @@ public final class SimpleTextCodec exten
   private final SegmentInfoFormat segmentInfos = new SimpleTextSegmentInfoFormat();
   private final FieldInfosFormat fieldInfosFormat = new SimpleTextFieldInfosFormat();
   private final TermVectorsFormat vectorsFormat = new SimpleTextTermVectorsFormat();
-  // TODO: need a plain-text impl
-  private final DocValuesFormat docValues = new SimpleTextDocValuesFormat();
-  // TODO: need a plain-text impl (using the above)
-  private final NormsFormat normsFormat = new SimpleTextNormsFormat();
   private final SimpleNormsFormat simpleNormsFormat = new SimpleTextSimpleNormsFormat();
   private final LiveDocsFormat liveDocs = new SimpleTextLiveDocsFormat();
 
@@ -61,11 +55,6 @@ public final class SimpleTextCodec exten
   }
 
   @Override
-  public DocValuesFormat docValuesFormat() {
-    return docValues;
-  }
-
-  @Override
   public StoredFieldsFormat storedFieldsFormat() {
     return storedFields;
   }
@@ -86,11 +75,6 @@ public final class SimpleTextCodec exten
   }
 
   @Override
-  public NormsFormat normsFormat() {
-    return normsFormat;
-  }
-
-  @Override
   public SimpleNormsFormat simpleNormsFormat() {
     return simpleNormsFormat;
   }

Modified: lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java (original)
+++ lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java Tue Jan 15 21:00:39 2013
@@ -25,10 +25,10 @@ import java.util.Map;
 import org.apache.lucene.codecs.FieldInfosReader;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -97,12 +97,12 @@ public class SimpleTextFieldInfosReader 
         SimpleTextUtil.readLine(input, scratch);
         assert StringHelper.startsWith(scratch, NORMS_TYPE);
         String nrmType = readString(NORMS_TYPE.length, scratch);
-        final DocValues.Type normsType = docValuesType(nrmType);
+        final DocValuesType normsType = docValuesType(nrmType);
         
         SimpleTextUtil.readLine(input, scratch);
         assert StringHelper.startsWith(scratch, DOCVALUES);
         String dvType = readString(DOCVALUES.length, scratch);
-        final DocValues.Type docValuesType = docValuesType(dvType);
+        final DocValuesType docValuesType = docValuesType(dvType);
         
         SimpleTextUtil.readLine(input, scratch);
         assert StringHelper.startsWith(scratch, NUM_ATTS);
@@ -140,11 +140,11 @@ public class SimpleTextFieldInfosReader 
     }
   }
 
-  public DocValues.Type docValuesType(String dvType) {
+  public DocValuesType docValuesType(String dvType) {
     if ("false".equals(dvType)) {
       return null;
     } else {
-      return DocValues.Type.valueOf(dvType);
+      return DocValuesType.valueOf(dvType);
     }
   }
   

Modified: lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosWriter.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosWriter.java (original)
+++ lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosWriter.java Tue Jan 15 21:00:39 2013
@@ -20,8 +20,8 @@ import java.io.IOException;
 import java.util.Map;
 
 import org.apache.lucene.codecs.FieldInfosWriter;
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
@@ -137,7 +137,7 @@ public class SimpleTextFieldInfosWriter 
     }
   }
   
-  private static String getDocValuesType(DocValues.Type type) {
+  private static String getDocValuesType(DocValuesType type) {
     return type == null ? "false" : type.toString();
   }
 }

Modified: lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSimpleDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSimpleDocValuesFormat.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSimpleDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSimpleDocValuesFormat.java Tue Jan 15 21:00:39 2013
@@ -34,8 +34,8 @@ import org.apache.lucene.codecs.SimpleDV
 import org.apache.lucene.codecs.SimpleDocValuesFormat;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.CorruptIndexException;
-import org.apache.lucene.index.DocValues;
 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;
@@ -161,8 +161,9 @@ public class SimpleTextSimpleDocValuesFo
     @Override
     public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
       assert fieldSeen(field.name);
-      assert (field.getDocValuesType() != null && (DocValues.isNumber(field.getDocValuesType()) || DocValues.isFloat(field.getDocValuesType()))) ||
-        (field.getNormType() != null && (DocValues.isNumber(field.getNormType()) || DocValues.isFloat(field.getNormType()))): "field=" + field.name;
+      // nocommit: this must be multiple asserts
+      //assert (field.getDocValuesType() != null && (DocValues.isNumber(field.getDocValuesType()) || DocValues.isFloat(field.getDocValuesType()))) ||
+      //  (field.getNormType() != null && (DocValues.isNumber(field.getNormType()) || DocValues.isFloat(field.getNormType()))): "field=" + field.name;
       writeFieldEntry(field);
 
       // first pass to find min/max
@@ -220,7 +221,7 @@ public class SimpleTextSimpleDocValuesFo
     @Override
     public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
       assert fieldSeen(field.name);
-      assert DocValues.isBytes(field.getDocValuesType());
+      assert field.getDocValuesType() == DocValuesType.BINARY;
       assert !isNorms;
       int maxLength = 0;
       for(BytesRef value : values) {
@@ -269,7 +270,7 @@ public class SimpleTextSimpleDocValuesFo
     @Override
     public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
       assert fieldSeen(field.name);
-      assert DocValues.isSortedBytes(field.getDocValuesType());
+      assert field.getDocValuesType() == DocValuesType.SORTED;
       assert !isNorms;
       writeFieldEntry(field);
 
@@ -421,9 +422,9 @@ public class SimpleTextSimpleDocValuesFo
         //System.out.println("  field=" + fieldName);
 
         // nocommit hack hack hack!!:
-        DocValues.Type dvType = ext.equals("slen") ? DocValues.Type.FIXED_INTS_8 : fieldInfo.getDocValuesType();
+        DocValuesType dvType = ext.equals("slen") ? DocValuesType.NUMERIC : fieldInfo.getDocValuesType();
         assert dvType != null;
-        if (DocValues.isNumber(dvType) || DocValues.isFloat(dvType)) {
+        if (dvType == DocValuesType.NUMERIC) {
           readLine();
           assert startsWith(MINVALUE): "got " + scratch.utf8ToString() + " field=" + fieldName + " ext=" + ext;
           field.minValue = Long.parseLong(stripPrefix(MINVALUE));
@@ -432,7 +433,7 @@ public class SimpleTextSimpleDocValuesFo
           field.pattern = stripPrefix(PATTERN);
           field.dataStartFilePointer = data.getFilePointer();
           data.seek(data.getFilePointer() + (1+field.pattern.length()) * maxDoc);
-        } else if (DocValues.isBytes(dvType)) {
+        } else if (dvType == DocValuesType.BINARY) {
           readLine();
           assert startsWith(MAXLENGTH);
           field.maxLength = Integer.parseInt(stripPrefix(MAXLENGTH));
@@ -441,7 +442,7 @@ public class SimpleTextSimpleDocValuesFo
           field.pattern = stripPrefix(PATTERN);
           field.dataStartFilePointer = data.getFilePointer();
           data.seek(data.getFilePointer() + (9+field.pattern.length()+field.maxLength) * maxDoc);
-        } else if (DocValues.isSortedBytes(dvType)) {
+        } else if (dvType == DocValuesType.SORTED) {
           readLine();
           assert startsWith(NUMVALUES);
           field.numValues = Integer.parseInt(stripPrefix(NUMVALUES));

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/Codec.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/Codec.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/Codec.java Tue Jan 15 21:00:39 2013
@@ -64,11 +64,8 @@ public abstract class Codec implements N
   
   /** Encodes/decodes postings */
   public abstract PostingsFormat postingsFormat();
-  
-  /** Encodes/decodes docvalues */
-  public abstract DocValuesFormat docValuesFormat();
 
-  /** Encodes/decodes streaming docvalues */
+  /** Encodes/decodes docvalues */
   public abstract SimpleDocValuesFormat simpleDocValuesFormat();
   
   /** Encodes/decodes stored fields */
@@ -84,12 +81,7 @@ public abstract class Codec implements N
   public abstract SegmentInfoFormat segmentInfoFormat();
   
   /** Encodes/decodes document normalization values */
-  public abstract NormsFormat normsFormat();
-  
-  public SimpleNormsFormat simpleNormsFormat() {
-    // nocommit make this abstract
-    return null;
-  }
+  public abstract SimpleNormsFormat simpleNormsFormat();
 
   /** Encodes/decodes live docs */
   public abstract LiveDocsFormat liveDocsFormat();

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java Tue Jan 15 21:00:39 2013
@@ -60,11 +60,6 @@ public abstract class FilterCodec extend
   }
 
   @Override
-  public DocValuesFormat docValuesFormat() {
-    return delegate.docValuesFormat();
-  }
-
-  @Override
   public SimpleDocValuesFormat simpleDocValuesFormat() {
     return delegate.simpleDocValuesFormat();
   }
@@ -80,11 +75,6 @@ public abstract class FilterCodec extend
   }
 
   @Override
-  public NormsFormat normsFormat() {
-    return delegate.normsFormat();
-  }
-
-  @Override
   public SimpleNormsFormat simpleNormsFormat() {
     return delegate.simpleNormsFormat();
   }

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java Tue Jan 15 21:00:39 2013
@@ -18,11 +18,9 @@ package org.apache.lucene.codecs.lucene4
  */
 
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FieldInfosFormat;
 import org.apache.lucene.codecs.FilterCodec;
 import org.apache.lucene.codecs.LiveDocsFormat;
-import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.SegmentInfoFormat;
 import org.apache.lucene.codecs.SimpleDocValuesFormat;
@@ -49,9 +47,7 @@ public final class Lucene40Codec extends
   private final StoredFieldsFormat fieldsFormat = new Lucene40StoredFieldsFormat();
   private final TermVectorsFormat vectorsFormat = new Lucene40TermVectorsFormat();
   private final FieldInfosFormat fieldInfosFormat = new Lucene40FieldInfosFormat();
-  private final DocValuesFormat docValuesFormat = new Lucene40DocValuesFormat();
   private final SegmentInfoFormat infosFormat = new Lucene40SegmentInfoFormat();
-  private final NormsFormat normsFormat = new Lucene40NormsFormat();
   private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
   
   private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
@@ -77,11 +73,6 @@ public final class Lucene40Codec extends
   }
 
   @Override
-  public final DocValuesFormat docValuesFormat() {
-    return docValuesFormat;
-  }
-
-  @Override
   public final PostingsFormat postingsFormat() {
     return postingsFormat;
   }
@@ -95,11 +86,6 @@ public final class Lucene40Codec extends
   public final SegmentInfoFormat segmentInfoFormat() {
     return infosFormat;
   }
-
-  @Override
-  public final NormsFormat normsFormat() {
-    return normsFormat;
-  }
   
   // nocommit need a read-only Lucene40SimpleDVFormat
   private final SimpleDocValuesFormat defaultDVFormat = SimpleDocValuesFormat.forName("Disk");

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java Tue Jan 15 21:00:39 2013
@@ -18,21 +18,17 @@ package org.apache.lucene.codecs.lucene4
  */
 
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FieldInfosFormat;
 import org.apache.lucene.codecs.FilterCodec;
 import org.apache.lucene.codecs.LiveDocsFormat;
-import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.SegmentInfoFormat;
 import org.apache.lucene.codecs.SimpleDocValuesFormat;
 import org.apache.lucene.codecs.SimpleNormsFormat;
 import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.TermVectorsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat;
 import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat;
 import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40NormsFormat;
 import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat;
 import org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat;
 import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
@@ -54,9 +50,7 @@ public class Lucene41Codec extends Codec
   private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
   private final TermVectorsFormat vectorsFormat = new Lucene40TermVectorsFormat();
   private final FieldInfosFormat fieldInfosFormat = new Lucene40FieldInfosFormat();
-  private final DocValuesFormat docValuesFormat = new Lucene40DocValuesFormat();
   private final SegmentInfoFormat infosFormat = new Lucene40SegmentInfoFormat();
-  private final NormsFormat normsFormat = new Lucene40NormsFormat();
   private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
   
   private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
@@ -90,11 +84,6 @@ public class Lucene41Codec extends Codec
   }
 
   @Override
-  public final DocValuesFormat docValuesFormat() {
-    return docValuesFormat;
-  }
-
-  @Override
   public final PostingsFormat postingsFormat() {
     return postingsFormat;
   }
@@ -108,11 +97,6 @@ public class Lucene41Codec extends Codec
   public final SegmentInfoFormat segmentInfoFormat() {
     return infosFormat;
   }
-
-  @Override
-  public final NormsFormat normsFormat() {
-    return normsFormat;
-  }
   
   @Override
   public final LiveDocsFormat liveDocsFormat() {

Added: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/BinaryDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/BinaryDocValuesField.java?rev=1433646&view=auto
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/BinaryDocValuesField.java (added)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/BinaryDocValuesField.java Tue Jan 15 21:00:39 2013
@@ -0,0 +1,35 @@
+package org.apache.lucene.document;
+
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.util.BytesRef;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+public class BinaryDocValuesField extends StoredField {
+  
+  public static final FieldType TYPE = new FieldType();
+  static {
+    TYPE.setDocValueType(FieldInfo.DocValuesType.BINARY);
+    TYPE.freeze();
+  }
+  
+  public BinaryDocValuesField(String name, BytesRef value) {
+    super(name, TYPE);
+    fieldsData = value;
+  }
+}

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/ByteDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/ByteDocValuesField.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/ByteDocValuesField.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/ByteDocValuesField.java Tue Jan 15 21:00:39 2013
@@ -17,7 +17,7 @@ package org.apache.lucene.document;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.FieldInfo;
 
 /**
  * <p>
@@ -32,7 +32,6 @@ import org.apache.lucene.index.DocValues
  * If you also need to store the value, you should add a
  * separate {@link StoredField} instance.
  * 
- * @see DocValues
  * */
 
 public class ByteDocValuesField extends StoredField {
@@ -42,7 +41,7 @@ public class ByteDocValuesField extends 
    */
   public static final FieldType TYPE = new FieldType();
   static {
-    TYPE.setDocValueType(DocValues.Type.FIXED_INTS_8);
+    TYPE.setDocValueType(FieldInfo.DocValuesType.NUMERIC);
     TYPE.freeze();
   }
 

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/DerefBytesDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/DerefBytesDocValuesField.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/DerefBytesDocValuesField.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/DerefBytesDocValuesField.java Tue Jan 15 21:00:39 2013
@@ -17,7 +17,6 @@ package org.apache.lucene.document;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.util.BytesRef;
 
 /**
@@ -38,29 +37,9 @@ import org.apache.lucene.util.BytesRef;
  * If you also need to store the value, you should add a
  * separate {@link StoredField} instance.
  * 
- * @see DocValues
  * */
-
-public class DerefBytesDocValuesField extends StoredField {
-
-  // TODO: ideally indexer figures out var vs fixed on its own!?
-  /**
-   * Type for indirect bytes DocValues: all with the same length
-   */
-  public static final FieldType TYPE_FIXED_LEN = new FieldType();
-  static {
-    TYPE_FIXED_LEN.setDocValueType(DocValues.Type.BYTES_FIXED_DEREF);
-    TYPE_FIXED_LEN.freeze();
-  }
-
-  /**
-   * Type for indirect bytes DocValues: can have variable lengths
-   */
-  public static final FieldType TYPE_VAR_LEN = new FieldType();
-  static {
-    TYPE_VAR_LEN.setDocValueType(DocValues.Type.BYTES_VAR_DEREF);
-    TYPE_VAR_LEN.freeze();
-  }
+@Deprecated
+public class DerefBytesDocValuesField extends SortedBytesDocValuesField {
 
   /**
    * Create a new variable-length indirect DocValues field.
@@ -75,8 +54,7 @@ public class DerefBytesDocValuesField ex
    * @throws IllegalArgumentException if the field name is null
    */
   public DerefBytesDocValuesField(String name, BytesRef bytes) {
-    super(name, TYPE_VAR_LEN);
-    fieldsData = bytes;
+    super(name, bytes);
   }
 
   /**
@@ -88,7 +66,6 @@ public class DerefBytesDocValuesField ex
    * @throws IllegalArgumentException if the field name is null
    */
   public DerefBytesDocValuesField(String name, BytesRef bytes, boolean isFixedLength) {
-    super(name, isFixedLength ? TYPE_FIXED_LEN : TYPE_VAR_LEN);
-    fieldsData = bytes;
+    super(name, bytes);
   }
 }

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/DoubleDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/DoubleDocValuesField.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/DoubleDocValuesField.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/DoubleDocValuesField.java Tue Jan 15 21:00:39 2013
@@ -17,7 +17,7 @@ package org.apache.lucene.document;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.FieldInfo;
 
 /**
  * <p>
@@ -32,7 +32,6 @@ import org.apache.lucene.index.DocValues
  * If you also need to store the value, you should add a
  * separate {@link StoredField} instance.
  * 
- * @see DocValues
  * */
 
 public class DoubleDocValuesField extends StoredField {
@@ -42,7 +41,7 @@ public class DoubleDocValuesField extend
    */
   public static final FieldType TYPE = new FieldType();
   static {
-    TYPE.setDocValueType(DocValues.Type.FLOAT_64);
+    TYPE.setDocValueType(FieldInfo.DocValuesType.NUMERIC);
     TYPE.freeze();
   }
 

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/Field.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/Field.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/Field.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/Field.java Tue Jan 15 21:00:39 2013
@@ -29,7 +29,6 @@ import org.apache.lucene.document.FieldT
 import org.apache.lucene.index.IndexWriter; // javadocs
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.IndexableFieldType;
-import org.apache.lucene.index.Norm; // javadocs
 import org.apache.lucene.index.StorableField;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.index.FieldInvertState; // javadocs

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/FieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/FieldType.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/FieldType.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/FieldType.java Tue Jan 15 21:00:39 2013
@@ -18,7 +18,7 @@ package org.apache.lucene.document;
  */
 
 import org.apache.lucene.analysis.Analyzer; // javadocs
-import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.IndexableFieldType;
 import org.apache.lucene.search.NumericRangeQuery; // javadocs
@@ -55,7 +55,7 @@ public class FieldType implements Indexa
   private NumericType numericType;
   private boolean frozen;
   private int numericPrecisionStep = NumericUtils.PRECISION_STEP_DEFAULT;
-  private DocValues.Type docValueType;
+  private DocValuesType docValueType;
 
   /**
    * Create a new mutable FieldType with all of the properties from <code>ref</code>
@@ -416,21 +416,21 @@ public class FieldType implements Indexa
    * {@inheritDoc}
    * <p>
    * The default is <code>null</code> (no docValues) 
-   * @see #setDocValueType(DocValues.Type)
+   * @see #setDocValueType(DocValuesType)
    */
   @Override
-  public DocValues.Type docValueType() {
+  public DocValuesType docValueType() {
     return docValueType;
   }
 
   /**
-   * Set's the field's DocValues.Type
+   * Set's the field's DocValuesType
    * @param type DocValues type, or null if no DocValues should be stored.
    * @throws IllegalStateException if this FieldType is frozen against
    *         future modifications.
    * @see #docValueType()
    */
-  public void setDocValueType(DocValues.Type type) {
+  public void setDocValueType(DocValuesType type) {
     checkIfFrozen();
     docValueType = type;
   }

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/FloatDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/FloatDocValuesField.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/FloatDocValuesField.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/FloatDocValuesField.java Tue Jan 15 21:00:39 2013
@@ -17,7 +17,7 @@ package org.apache.lucene.document;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.FieldInfo;
 
 /**
  * <p>
@@ -31,7 +31,6 @@ import org.apache.lucene.index.DocValues
  * <p>
  * If you also need to store the value, you should add a
  * separate {@link StoredField} instance.
- * @see DocValues
  * */
 
 public class FloatDocValuesField extends StoredField {
@@ -41,7 +40,7 @@ public class FloatDocValuesField extends
    */
   public static final FieldType TYPE = new FieldType();
   static {
-    TYPE.setDocValueType(DocValues.Type.FLOAT_32);
+    TYPE.setDocValueType(FieldInfo.DocValuesType.NUMERIC);
     TYPE.freeze();
   }
 

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/IntDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/IntDocValuesField.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/IntDocValuesField.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/IntDocValuesField.java Tue Jan 15 21:00:39 2013
@@ -17,7 +17,7 @@ package org.apache.lucene.document;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.FieldInfo;
 
 /**
  * <p>
@@ -31,7 +31,6 @@ import org.apache.lucene.index.DocValues
  * <p>
  * If you also need to store the value, you should add a
  * separate {@link StoredField} instance.
- * @see DocValues
  * */
 
 public class IntDocValuesField extends StoredField {
@@ -41,7 +40,7 @@ public class IntDocValuesField extends S
    */
   public static final FieldType TYPE = new FieldType();
   static {
-    TYPE.setDocValueType(DocValues.Type.FIXED_INTS_32);
+    TYPE.setDocValueType(FieldInfo.DocValuesType.NUMERIC);
     TYPE.freeze();
   }
 

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/LongDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/LongDocValuesField.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/LongDocValuesField.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/LongDocValuesField.java Tue Jan 15 21:00:39 2013
@@ -17,7 +17,7 @@ package org.apache.lucene.document;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.FieldInfo;
 
 /**
  * <p>
@@ -31,7 +31,6 @@ import org.apache.lucene.index.DocValues
  * <p>
  * If you also need to store the value, you should add a
  * separate {@link StoredField} instance.
- * @see DocValues
  * */
 
 public class LongDocValuesField extends StoredField {
@@ -41,7 +40,7 @@ public class LongDocValuesField extends 
    */
   public static final FieldType TYPE = new FieldType();
   static {
-    TYPE.setDocValueType(DocValues.Type.FIXED_INTS_64);
+    TYPE.setDocValueType(FieldInfo.DocValuesType.NUMERIC);
     TYPE.freeze();
   }
 

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/PackedLongDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/PackedLongDocValuesField.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/PackedLongDocValuesField.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/PackedLongDocValuesField.java Tue Jan 15 21:00:39 2013
@@ -17,7 +17,6 @@ package org.apache.lucene.document;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.AtomicReader;      // javadocs
 
 /**
@@ -25,7 +24,7 @@ import org.apache.lucene.index.AtomicRea
  * Field that stores a per-document <code>long</code> value 
  * for scoring, sorting or value retrieval.  The values are 
  * encoded in the index an in RAM (when loaded via 
- * {@link AtomicReader#docValues})
+ * {@link AtomicReader#getNumericDocValues(String)})
  * using packed ints. Here's an example usage:
  * 
  * <pre class="prettyprint">
@@ -36,19 +35,10 @@ import org.apache.lucene.index.AtomicRea
  * If you also need to store the value, you should add a
  * separate {@link StoredField} instance.
  * 
- * @see DocValues
  * */
 
-public class PackedLongDocValuesField extends StoredField {
-
-  /**
-   * Type for packed long DocValues.
-   */
-  public static final FieldType TYPE = new FieldType();
-  static {
-    TYPE.setDocValueType(DocValues.Type.VAR_INTS);
-    TYPE.freeze();
-  }
+@Deprecated
+public class PackedLongDocValuesField extends LongDocValuesField {
 
   /** 
    * Creates a new DocValues field with the specified long value 
@@ -57,7 +47,6 @@ public class PackedLongDocValuesField ex
    * @throws IllegalArgumentException if the field name is null
    */
   public PackedLongDocValuesField(String name, long value) {
-    super(name, TYPE);
-    fieldsData = Long.valueOf(value);
+    super(name, value);
   }
 }

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/ShortDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/ShortDocValuesField.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/ShortDocValuesField.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/ShortDocValuesField.java Tue Jan 15 21:00:39 2013
@@ -17,7 +17,7 @@ package org.apache.lucene.document;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.FieldInfo;
 
 /**
  * <p>
@@ -32,7 +32,6 @@ import org.apache.lucene.index.DocValues
  * If you also need to store the value, you should add a
  * separate {@link StoredField} instance.
  * 
- * @see DocValues
  * */
 
 public class ShortDocValuesField extends StoredField {
@@ -42,7 +41,7 @@ public class ShortDocValuesField extends
    */
   public static final FieldType TYPE = new FieldType();
   static {
-    TYPE.setDocValueType(DocValues.Type.FIXED_INTS_16);
+    TYPE.setDocValueType(FieldInfo.DocValuesType.NUMERIC);
     TYPE.freeze();
   }
 

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/SortedBytesDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/SortedBytesDocValuesField.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/SortedBytesDocValuesField.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/SortedBytesDocValuesField.java Tue Jan 15 21:00:39 2013
@@ -17,7 +17,7 @@ package org.apache.lucene.document;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.util.BytesRef;
 
 /**
@@ -34,44 +34,28 @@ import org.apache.lucene.util.BytesRef;
  * If you also need to store the value, you should add a
  * separate {@link StoredField} instance.
  * 
- * @see DocValues
  * */
 
 public class SortedBytesDocValuesField extends StoredField {
 
-  // TODO: ideally indexer figures out var vs fixed on its own!?
   /**
    * Type for sorted bytes DocValues: all with the same length
    */
-  public static final FieldType TYPE_FIXED_LEN = new FieldType();
+  public static final FieldType TYPE = new FieldType();
   static {
-    TYPE_FIXED_LEN.setDocValueType(DocValues.Type.BYTES_FIXED_SORTED);
-    TYPE_FIXED_LEN.freeze();
+    TYPE.setDocValueType(FieldInfo.DocValuesType.SORTED);
+    TYPE.freeze();
   }
 
   /**
-   * Type for sorted bytes DocValues: can have variable lengths
-   */
-  public static final FieldType TYPE_VAR_LEN = new FieldType();
-  static {
-    TYPE_VAR_LEN.setDocValueType(DocValues.Type.BYTES_VAR_SORTED);
-    TYPE_VAR_LEN.freeze();
-  }
-
-  /**
-   * Create a new variable-length sorted DocValues field.
-   * <p>
-   * This calls 
-   * {@link SortedBytesDocValuesField#SortedBytesDocValuesField(String, BytesRef, boolean)
-   *  SortedBytesDocValuesField(name, bytes, false}, meaning by default
-   * it allows for values of different lengths. If your values are all 
-   * the same length, use that constructor instead.
+   * Create a new sorted DocValues field.
    * @param name field name
    * @param bytes binary content
    * @throws IllegalArgumentException if the field name is null
    */
   public SortedBytesDocValuesField(String name, BytesRef bytes) {
-    this(name, bytes, false);
+    super(name, TYPE);
+    fieldsData = bytes;
   }
 
   /**
@@ -81,8 +65,9 @@ public class SortedBytesDocValuesField e
    * @param isFixedLength true if all values have the same length.
    * @throws IllegalArgumentException if the field name is null
    */
+  @Deprecated
   public SortedBytesDocValuesField(String name, BytesRef bytes, boolean isFixedLength) {
-    super(name, isFixedLength ? TYPE_FIXED_LEN : TYPE_VAR_LEN);
+    super(name, TYPE);
     fieldsData = bytes;
   }
 }

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/StraightBytesDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/StraightBytesDocValuesField.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/StraightBytesDocValuesField.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/document/StraightBytesDocValuesField.java Tue Jan 15 21:00:39 2013
@@ -17,7 +17,6 @@ package org.apache.lucene.document;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.util.BytesRef;
 
 /**
@@ -37,29 +36,11 @@ import org.apache.lucene.util.BytesRef;
  * If you also need to store the value, you should add a
  * separate {@link StoredField} instance.
  * 
- * @see DocValues
  * */
 
-public class StraightBytesDocValuesField extends StoredField {
+@Deprecated
+public class StraightBytesDocValuesField extends BinaryDocValuesField {
 
-  // TODO: ideally indexer figures out var vs fixed on its own!?
-  /**
-   * Type for direct bytes DocValues: all with the same length
-   */
-  public static final FieldType TYPE_FIXED_LEN = new FieldType();
-  static {
-    TYPE_FIXED_LEN.setDocValueType(DocValues.Type.BYTES_FIXED_STRAIGHT);
-    TYPE_FIXED_LEN.freeze();
-  }
-
-  /**
-   * Type for direct bytes DocValues: can have variable lengths
-   */
-  public static final FieldType TYPE_VAR_LEN = new FieldType();
-  static {
-    TYPE_VAR_LEN.setDocValueType(DocValues.Type.BYTES_VAR_STRAIGHT);
-    TYPE_VAR_LEN.freeze();
-  }
 
   /**
    * Create a new variable-length direct DocValues field.
@@ -74,8 +55,7 @@ public class StraightBytesDocValuesField
    * @throws IllegalArgumentException if the field name is null
    */
   public StraightBytesDocValuesField(String name, BytesRef bytes) {
-    super(name, TYPE_VAR_LEN);
-    fieldsData = bytes;
+    super(name, bytes);
   }
 
   /**
@@ -87,7 +67,6 @@ public class StraightBytesDocValuesField
    * @throws IllegalArgumentException if the field name is null
    */
   public StraightBytesDocValuesField(String name, BytesRef bytes, boolean isFixedLength) {
-    super(name, isFixedLength ? TYPE_FIXED_LEN : TYPE_VAR_LEN);
-    fieldsData = bytes;
+    super(name, bytes);
   }
 }

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java Tue Jan 15 21:00:39 2013
@@ -156,13 +156,6 @@ public abstract class AtomicReader exten
     }
     return null;
   }
-  
-  /**
-   * Returns {@link DocValues} for this field.
-   * This method may return null if the reader has no per-document
-   * values stored.
-   */
-  public abstract DocValues docValues(String field) throws IOException;
 
   /** Returns {@link NumericDocValues} for this field, or
    *  null if no {@link NumericDocValues} were indexed for
@@ -182,12 +175,6 @@ public abstract class AtomicReader exten
    *  used by a single thread. */
   public abstract SortedDocValues getSortedDocValues(String field) throws IOException;
 
-  /**
-   * Returns {@link DocValues} for this field's normalization values.
-   * This method may return null if the field has no norms.
-   */
-  public abstract DocValues normValues(String field) throws IOException;
-
   // nocommit document that these are thread-private:
   /** Returns {@link NumericDocValues} representing norms
    *  for this field, or null if no {@link NumericDocValues}

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Tue Jan 15 21:00:39 2013
@@ -32,8 +32,6 @@ import org.apache.lucene.codecs.BlockTre
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.PostingsFormat; // javadocs
 import org.apache.lucene.document.FieldType; // for javadocs
-import org.apache.lucene.index.DocValues.SortedSource;
-import org.apache.lucene.index.DocValues.Source;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.Directory;
@@ -678,12 +676,10 @@ public class CheckIndex {
       }
       for (FieldInfo info : reader.getFieldInfos()) {
         if (info.hasNorms()) {
-          DocValues dv = reader.normValues(info.name);
-          checkDocValues(dv, info.name, info.getNormType(), reader.maxDoc());
           checkSimpleNorms(info, reader, infoStream);
           ++status.totFields;
         } else {
-          if (reader.normValues(info.name) != null) {
+          if (reader.simpleNormValues(info.name) != null) {
             throw new RuntimeException("field: " + info.name + " should omit norms but has them!");
           }
         }
@@ -1256,92 +1252,6 @@ public class CheckIndex {
     return status;
   }
   
-  /** Helper method to verify values (either docvalues or norms), also checking
-   *  type and size against fieldinfos/segmentinfo
-   */
-  private static void checkDocValues(DocValues docValues, String fieldName, DocValues.Type expectedType, int expectedDocs) throws IOException {
-    if (docValues == null) {
-      throw new RuntimeException("field: " + fieldName + " omits docvalues but should have them!");
-    }
-    DocValues.Type type = docValues.getType();
-    if (type != expectedType) {
-      throw new RuntimeException("field: " + fieldName + " has type: " + type + " but fieldInfos says:" + expectedType);
-    }
-    final Source values = docValues.getDirectSource();
-    int size = docValues.getValueSize();
-    for (int i = 0; i < expectedDocs; i++) {
-      switch (type) {
-      case BYTES_FIXED_SORTED:
-      case BYTES_VAR_SORTED:
-      case BYTES_FIXED_DEREF:
-      case BYTES_FIXED_STRAIGHT:
-      case BYTES_VAR_DEREF:
-      case BYTES_VAR_STRAIGHT:
-        BytesRef bytes = new BytesRef();
-        values.getBytes(i, bytes);
-        if (size != -1 && size != bytes.length) {
-          throw new RuntimeException("field: " + fieldName + " returned wrongly sized bytes, was: " + bytes.length + " should be: " + size);
-        }
-        break;
-      case FLOAT_32:
-        assert size == 4;
-        values.getFloat(i);
-        break;
-      case FLOAT_64:
-        assert size == 8;
-        values.getFloat(i);
-        break;
-      case VAR_INTS:
-        assert size == -1;
-        values.getInt(i);
-        break;
-      case FIXED_INTS_16:
-        assert size == 2;
-        values.getInt(i);
-        break;
-      case FIXED_INTS_32:
-        assert size == 4;
-        values.getInt(i);
-        break;
-      case FIXED_INTS_64:
-        assert size == 8;
-        values.getInt(i);
-        break;
-      case FIXED_INTS_8:
-        assert size == 1;
-        values.getInt(i);
-        break;
-      default:
-        throw new IllegalArgumentException("Field: " + fieldName
-                    + " - no such DocValues type: " + type);
-      }
-    }
-    if (type == DocValues.Type.BYTES_FIXED_SORTED || type == DocValues.Type.BYTES_VAR_SORTED) {
-      // check sorted bytes
-      SortedSource sortedValues = values.asSortedSource();
-      Comparator<BytesRef> comparator = sortedValues.getComparator();
-      int lastOrd = -1;
-      BytesRef lastBytes = new BytesRef();
-      for (int i = 0; i < expectedDocs; i++) {
-        int ord = sortedValues.ord(i);
-        if (ord < 0 || ord > expectedDocs) {
-          throw new RuntimeException("field: " + fieldName + " ord is out of bounds: " + ord);
-        }
-        BytesRef bytes = new BytesRef();
-        sortedValues.getByOrd(ord, bytes);
-        if (lastOrd != -1) {
-          int ordComp = Integer.signum(new Integer(ord).compareTo(new Integer(lastOrd)));
-          int bytesComp = Integer.signum(comparator.compare(bytes, lastBytes));
-          if (ordComp != bytesComp) {
-            throw new RuntimeException("field: " + fieldName + " ord comparison is wrong: " + ordComp + " comparator claims: " + bytesComp);
-          }
-        }
-        lastOrd = ord;
-        lastBytes = bytes;
-      }
-    }
-  }
-  
   public static Status.DocValuesStatus testDocValues(AtomicReader reader,
                                                      PrintStream infoStream) {
     final Status.DocValuesStatus status = new Status.DocValuesStatus();
@@ -1352,11 +1262,11 @@ public class CheckIndex {
       for (FieldInfo fieldInfo : reader.getFieldInfos()) {
         if (fieldInfo.hasDocValues()) {
           status.totalValueFields++;
-          final DocValues docValues = reader.docValues(fieldInfo.name);
-          checkDocValues(docValues, fieldInfo.name, fieldInfo.getDocValuesType(), reader.maxDoc());
           checkSimpleDocValues(fieldInfo, reader, infoStream);
         } else {
-          if (reader.docValues(fieldInfo.name) != null) {
+          if (reader.getBinaryDocValues(fieldInfo.name) != null ||
+              reader.getNumericDocValues(fieldInfo.name) != null ||
+              reader.getSortedDocValues(fieldInfo.name) != null) {
             throw new RuntimeException("field: " + fieldInfo.name + " has docvalues but should omit them!");
           }
         }
@@ -1440,23 +1350,13 @@ public class CheckIndex {
       msg(infoStream, "  field: " + fi.name + ": " + atts);
     }
     switch(fi.getDocValuesType()) {
-      case BYTES_FIXED_SORTED:
-      case BYTES_VAR_SORTED:
-      case BYTES_FIXED_DEREF:
-      case BYTES_VAR_DEREF:
+      case SORTED:
         checkSortedDocValues(fi.name, reader, reader.getSortedDocValues(fi.name));
         break;
-      case BYTES_FIXED_STRAIGHT:
-      case BYTES_VAR_STRAIGHT:
+      case BINARY:
         checkBinaryDocValues(fi.name, reader, reader.getBinaryDocValues(fi.name));
         break;
-      case FLOAT_32:
-      case FLOAT_64:
-      case VAR_INTS:
-      case FIXED_INTS_16:
-      case FIXED_INTS_32:
-      case FIXED_INTS_64:
-      case FIXED_INTS_8:
+      case NUMERIC:
         checkNumericDocValues(fi.name, reader, reader.getNumericDocValues(fi.name));
         break;
       default:
@@ -1467,13 +1367,7 @@ public class CheckIndex {
   // nocommit
   public static void checkSimpleNorms(FieldInfo fi, AtomicReader reader, PrintStream infoStream) throws IOException {
     switch(fi.getNormType()) {
-      case FLOAT_32:
-      case FLOAT_64:
-      case VAR_INTS:
-      case FIXED_INTS_16:
-      case FIXED_INTS_32:
-      case FIXED_INTS_64:
-      case FIXED_INTS_8:
+      case NUMERIC:
         checkNumericDocValues(fi.name, reader, reader.simpleNormValues(fi.name));
         break;
       default:

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java Tue Jan 15 21:00:39 2013
@@ -25,16 +25,10 @@ import java.util.HashSet;
 import java.util.Map;
 
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.FieldInfosWriter;
-import org.apache.lucene.codecs.PerDocConsumer;
-import org.apache.lucene.index.DocumentsWriterPerThread.DocState;
-import org.apache.lucene.index.TypePromoter.TypeCompatibility;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Counter;
-import org.apache.lucene.util.IOUtils;
-
 
 /**
  * This is a DocConsumer that gathers all fields under the
@@ -86,14 +80,6 @@ final class DocFieldProcessor extends Do
     storedConsumer.flush(state);
     consumer.flush(childFields, state);
 
-    for (DocValuesConsumerHolder consumer : docValues.values()) {
-      consumer.docValuesConsumer.finish(state.segmentInfo.getDocCount());
-    }
-    
-    // close perDocConsumer during flush to ensure all files are flushed due to PerCodec CFS
-    // nocommit
-    IOUtils.close(perDocConsumer);
-
     // Important to save after asking consumer to flush so
     // consumer can alter the FieldInfo* if necessary.  EG,
     // FreqProxTermsWriter does this with
@@ -119,8 +105,6 @@ final class DocFieldProcessor extends Do
         field = next;
       }
     }
-    IOUtils.closeWhileHandlingException(perDocConsumer);
-    // TODO add abort to PerDocConsumer!
     
     try {
       storedConsumer.abort();
@@ -138,16 +122,6 @@ final class DocFieldProcessor extends Do
       }
     }
     
-    try {
-      if (perDocConsumer != null) {
-        perDocConsumer.abort();  
-      }
-    } catch (Throwable t) {
-      if (th == null) {
-        th = t;
-      }
-    }
-    
     // If any errors occured, throw it.
     if (th != null) {
       if (th instanceof RuntimeException) throw (RuntimeException) th;
@@ -177,8 +151,6 @@ final class DocFieldProcessor extends Do
     fieldHash = new DocFieldProcessorPerField[2];
     hashMask = 1;
     totalFieldCount = 0;
-    perDocConsumer = null;
-    docValues.clear();
   }
 
   private void rehash() {
@@ -233,30 +205,6 @@ final class DocFieldProcessor extends Do
       IndexableFieldType ft = field.fieldType();
       FieldInfo fieldInfo = fieldInfos.addOrUpdate(fieldName, ft);
       storedConsumer.addField(docState.docID, field, fieldInfo);
-      
-      final DocValues.Type dvType = ft.docValueType();
-      if (dvType != null) {
-        DocValuesConsumerHolder docValuesConsumer = docValuesConsumer(dvType,
-            docState, fieldInfo);
-        DocValuesConsumer consumer = docValuesConsumer.docValuesConsumer;
-        if (docValuesConsumer.compatibility == null) {
-          consumer.add(docState.docID, field);
-          docValuesConsumer.compatibility = new TypeCompatibility(dvType,
-              consumer.getValueSize());
-        } else if (docValuesConsumer.compatibility.isCompatible(dvType,
-            TypePromoter.getValueSize(dvType, field.binaryValue()))) {
-          consumer.add(docState.docID, field);
-        } else {
-          docValuesConsumer.compatibility.isCompatible(dvType,
-              TypePromoter.getValueSize(dvType, field.binaryValue()));
-          TypeCompatibility compatibility = docValuesConsumer.compatibility;
-          throw new IllegalArgumentException("Incompatible DocValues type: "
-              + dvType.name() + " size: "
-              + TypePromoter.getValueSize(dvType, field.binaryValue())
-              + " expected: " + " type: " + compatibility.getBaseType()
-              + " size: " + compatibility.getBaseSize());
-        }
-      }
     }
 
     // If we are writing vectors then we must visit
@@ -344,45 +292,4 @@ final class DocFieldProcessor extends Do
       consumer.finishDocument();
     }
   }
-
-  private static class DocValuesConsumerHolder {
-    // Only used to enforce that same DV field name is never
-    // added more than once per doc:
-    int docID;
-    final DocValuesConsumer docValuesConsumer;
-    TypeCompatibility compatibility;
-
-    public DocValuesConsumerHolder(DocValuesConsumer docValuesConsumer) {
-      this.docValuesConsumer = docValuesConsumer;
-    }
-  }
-
-  final private Map<String, DocValuesConsumerHolder> docValues = new HashMap<String, DocValuesConsumerHolder>();
-  private PerDocConsumer perDocConsumer;
-
-  DocValuesConsumerHolder docValuesConsumer(DocValues.Type valueType, DocState docState, FieldInfo fieldInfo) 
-      throws IOException {
-    DocValuesConsumerHolder docValuesConsumerAndDocID = docValues.get(fieldInfo.name);
-    if (docValuesConsumerAndDocID != null) {
-      if (docState.docID == docValuesConsumerAndDocID.docID) {
-        throw new IllegalArgumentException("DocValuesField \"" + fieldInfo.name + "\" appears more than once in this document (only one value is allowed, per field)");
-      }
-      assert docValuesConsumerAndDocID.docID < docState.docID;
-      docValuesConsumerAndDocID.docID = docState.docID;
-      return docValuesConsumerAndDocID;
-    }
-
-    if (perDocConsumer == null) {
-      PerDocWriteState perDocWriteState = docState.docWriter.newPerDocWriteState("");
-      perDocConsumer = docState.docWriter.codec.docValuesFormat().docsConsumer(perDocWriteState);
-      if (perDocConsumer == null) {
-        throw new IllegalStateException("codec=" +  docState.docWriter.codec + " does not support docValues: from docValuesFormat().docsConsumer(...) returned null; field=" + fieldInfo.name);
-      }
-    }
-    DocValuesConsumer docValuesConsumer = perDocConsumer.addValuesField(valueType, fieldInfo);
-    docValuesConsumerAndDocID = new DocValuesConsumerHolder(docValuesConsumer);
-    docValuesConsumerAndDocID.docID = docState.docID;
-    docValues.put(fieldInfo.name, docValuesConsumerAndDocID);
-    return docValuesConsumerAndDocID;
-  }
 }

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/DocValuesProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/DocValuesProcessor.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/DocValuesProcessor.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/DocValuesProcessor.java Tue Jan 15 21:00:39 2013
@@ -23,6 +23,7 @@ import java.util.Map;
 
 import org.apache.lucene.codecs.SimpleDVConsumer;
 import org.apache.lucene.codecs.SimpleDocValuesFormat;
+import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Counter;
 
@@ -47,27 +48,25 @@ final class DocValuesProcessor extends S
 
   @Override
   public void addField(int docID, StorableField field, FieldInfo fieldInfo) {
-    final DocValues.Type dvType = field.fieldType().docValueType();
+    // nocommit: these checks are duplicated everywhere
+    final DocValuesType dvType = field.fieldType().docValueType();
     if (dvType != null) {
-      DocValues.Type currentDVType = fieldInfo.getDocValuesType();
+      DocValuesType currentDVType = fieldInfo.getDocValuesType();
       if (currentDVType == null) {
         fieldInfo.setDocValuesType(dvType);
       } else if (currentDVType != dvType) {
         throw new IllegalArgumentException("cannot change DocValues type from " + currentDVType + " to " + dvType + " for field \"" + fieldInfo.name + "\"");
       }
-      if (DocValues.isBytes(dvType)) {
+      if (dvType == DocValuesType.BINARY) {
         addBinaryField(fieldInfo, docID, field.binaryValue());
-      } else if (DocValues.isSortedBytes(dvType)) {
+      } else if (dvType == DocValuesType.SORTED) {
         addSortedField(fieldInfo, docID, field.binaryValue());
-      } else if (DocValues.isFloat(dvType)) {
-        if (dvType == DocValues.Type.FLOAT_32) {
-          addNumericField(fieldInfo, docID, field.numericValue().floatValue());
-        } else if (dvType == DocValues.Type.FLOAT_64) {
-          addNumericField(fieldInfo, docID, field.numericValue().doubleValue());
-        } else {
-          assert false;
-        }
-      } else if (DocValues.isNumber(dvType)) {
+        // nocommit: hack
+      } else if (dvType == DocValuesType.NUMERIC && field.numericValue() instanceof Float) {
+        addNumericField(fieldInfo, docID, field.numericValue().floatValue());
+      } else if (dvType == DocValuesType.NUMERIC && field.numericValue() instanceof Double) {
+        addNumericField(fieldInfo, docID, field.numericValue().doubleValue());
+      } else if (dvType == DocValuesType.NUMERIC) {
         addNumericField(fieldInfo, docID, field.numericValue().longValue());
       } else {
         assert false: "unrecognized DocValues.Type: " + dvType;

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Tue Jan 15 21:00:39 2013
@@ -87,7 +87,7 @@ class DocumentsWriterPerThread {
 
       final InvertedDocConsumer termsHash = new TermsHash(documentsWriterPerThread, freqProxWriter, true,
                                                           new TermsHash(documentsWriterPerThread, termVectorsWriter, false, null));
-      final NormsConsumer normsWriter = new NormsConsumer(documentsWriterPerThread);
+      final NormsConsumer normsWriter = new NormsConsumer();
       final DocInverter docInverter = new DocInverter(documentsWriterPerThread.docState, termsHash, normsWriter);
       final StoredFieldsConsumer storedFields = new TwoStoredFieldsConsumers(
                                                       new StoredFieldsProcessor(documentsWriterPerThread),
@@ -657,10 +657,6 @@ class DocumentsWriterPerThread {
     }
     
   }
-  PerDocWriteState newPerDocWriteState(String segmentSuffix) {
-    assert segmentInfo != null;
-    return new PerDocWriteState(infoStream, directory, segmentInfo, bytesUsed, segmentSuffix, IOContext.DEFAULT);
-  }
   
   @Override
   public String toString() {

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java Tue Jan 15 21:00:39 2013
@@ -20,8 +20,6 @@ package org.apache.lucene.index;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.lucene.index.DocValues.Type;
-
 /**
  *  Access to the Field Info file that describes document fields and whether or
  *  not they are indexed. Each segment has a separate Field Info file. Objects
@@ -37,12 +35,12 @@ public final class FieldInfo {
   public final int number;
 
   private boolean indexed;
-  private DocValues.Type docValueType;
+  private DocValuesType docValueType;
 
   // True if any document indexed term vectors
   private boolean storeTermVector;
 
-  private DocValues.Type normType;
+  private DocValuesType normType;
   private boolean omitNorms; // omit norms associated with indexed fields  
   private IndexOptions indexOptions;
   private boolean storePayloads; // whether this field stores payloads together with term positions
@@ -82,6 +80,12 @@ public final class FieldInfo {
      */
     DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS,
   };
+  
+  public static enum DocValuesType {
+    NUMERIC,
+    BINARY,
+    SORTED
+  };
 
   /**
    * Sole Constructor.
@@ -89,7 +93,7 @@ public final class FieldInfo {
    * @lucene.experimental
    */
   public FieldInfo(String name, boolean indexed, int number, boolean storeTermVector, 
-            boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValues.Type docValues, DocValues.Type normsType, Map<String,String> attributes) {
+            boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValuesType docValues, DocValuesType normsType, Map<String,String> attributes) {
     this.name = name;
     this.indexed = indexed;
     this.number = number;
@@ -163,7 +167,7 @@ public final class FieldInfo {
     assert checkConsistency();
   }
 
-  void setDocValuesType(DocValues.Type type) {
+  void setDocValuesType(DocValuesType type) {
     docValueType = type;
     assert checkConsistency();
   }
@@ -181,16 +185,16 @@ public final class FieldInfo {
   }
 
   /**
-   * Returns {@link DocValues.Type} of the docValues. this may be null if the field has no docvalues.
+   * Returns {@link DocValuesType} of the docValues. this may be null if the field has no docvalues.
    */
-  public DocValues.Type getDocValuesType() {
+  public DocValuesType getDocValuesType() {
     return docValueType;
   }
   
   /**
-   * Returns {@link DocValues.Type} of the norm. this may be null if the field has no norms.
+   * Returns {@link DocValuesType} of the norm. this may be null if the field has no norms.
    */
-  public DocValues.Type getNormType() {
+  public DocValuesType getNormType() {
     return normType;
   }
 
@@ -206,7 +210,7 @@ public final class FieldInfo {
     assert checkConsistency();
   }
 
-  void setNormValueType(Type type) {
+  void setNormValueType(DocValuesType type) {
     normType = type;
     assert checkConsistency();
   }

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java Tue Jan 15 21:00:39 2013
@@ -25,6 +25,7 @@ import java.util.Map;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
+import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 
 /** 
@@ -162,13 +163,13 @@ public class FieldInfos implements Itera
     
     private final Map<Integer,String> numberToName;
     private final Map<String,Integer> nameToNumber;
-    private final Map<String,DocValues.Type> docValuesType;
+    private final Map<String,DocValuesType> docValuesType;
     private int lowestUnassignedFieldNumber = -1;
     
     FieldNumbers() {
       this.nameToNumber = new HashMap<String, Integer>();
       this.numberToName = new HashMap<Integer, String>();
-      this.docValuesType = new HashMap<String,DocValues.Type>();
+      this.docValuesType = new HashMap<String,DocValuesType>();
     }
     
     /**
@@ -177,9 +178,9 @@ public class FieldInfos implements Itera
      * number assigned if possible otherwise the first unassigned field number
      * is used as the field number.
      */
-    synchronized int addOrGet(String fieldName, int preferredFieldNumber, DocValues.Type dvType) {
+    synchronized int addOrGet(String fieldName, int preferredFieldNumber, DocValuesType dvType) {
       if (dvType != null) {
-        DocValues.Type currentDVType = docValuesType.get(fieldName);
+        DocValuesType currentDVType = docValuesType.get(fieldName);
         if (currentDVType == null) {
           docValuesType.put(fieldName, dvType);
         } else if (currentDVType != null && currentDVType != dvType) {
@@ -211,7 +212,8 @@ public class FieldInfos implements Itera
     /**
      * Sets the given field number and name if not yet set. 
      */
-    synchronized void setIfNotSet(int fieldNumber, String fieldName, DocValues.Type dvType) {
+    // nocommit: why is docvalues involved with global field numbers?
+    synchronized void setIfNotSet(int fieldNumber, String fieldName, DocValuesType dvType) {
       final Integer boxedFieldNumber = Integer.valueOf(fieldNumber);
       if (!numberToName.containsKey(boxedFieldNumber)
           && !nameToNumber.containsKey(fieldName)
@@ -225,7 +227,7 @@ public class FieldInfos implements Itera
     }
     
     // used by assert
-    synchronized boolean containsConsistent(Integer number, String name, DocValues.Type dvType) {
+    synchronized boolean containsConsistent(Integer number, String name, DocValuesType dvType) {
       return name.equals(numberToName.get(number))
           && number.equals(nameToNumber.get(name)) &&
         (dvType == null || docValuesType.get(name) == null || dvType == docValuesType.get(name));
@@ -285,7 +287,7 @@ public class FieldInfos implements Itera
      */
     // TODO: fix testCodecs to do this another way, its the only user of this
     FieldInfo addOrUpdate(String name, boolean isIndexed, boolean storeTermVector,
-                         boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValues.Type docValues, DocValues.Type normType) {
+                         boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValuesType docValues, DocValuesType normType) {
       return addOrUpdateInternal(name, -1, isIndexed, storeTermVector, omitNorms, storePayloads, indexOptions, docValues, normType);
     }
 
@@ -307,7 +309,7 @@ public class FieldInfos implements Itera
 
     private FieldInfo addOrUpdateInternal(String name, int preferredFieldNumber, boolean isIndexed,
         boolean storeTermVector,
-        boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValues.Type docValues, DocValues.Type normType) {
+        boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValuesType docValues, DocValuesType normType) {
       FieldInfo fi = fieldInfo(name);
       if (fi == null) {
         // get a global number for this field
@@ -317,7 +319,7 @@ public class FieldInfos implements Itera
         fi.update(isIndexed, storeTermVector, omitNorms, storePayloads, indexOptions);
 
         if (docValues != null) {
-          DocValues.Type currentDVType = fi.getDocValuesType();
+          DocValuesType currentDVType = fi.getDocValuesType();
           if (currentDVType == null) {
             fi.setDocValuesType(docValues);
           } else if (currentDVType != docValues) {
@@ -327,7 +329,7 @@ public class FieldInfos implements Itera
         }
 
         if (!fi.omitsNorms() && normType != null) {
-          DocValues.Type currentDVType = fi.getNormType();
+          DocValuesType currentDVType = fi.getNormType();
           if (currentDVType == null) {
             fi.setNormValueType(docValues);
           } else if (currentDVType != normType) {
@@ -348,7 +350,7 @@ public class FieldInfos implements Itera
     
     private FieldInfo addInternal(String name, int fieldNumber, boolean isIndexed,
                                   boolean storeTermVector, boolean omitNorms, boolean storePayloads,
-                                  IndexOptions indexOptions, DocValues.Type docValuesType, DocValues.Type normType) {
+                                  IndexOptions indexOptions, DocValuesType docValuesType, DocValuesType normType) {
       globalFieldNumbers.setIfNotSet(fieldNumber, name, docValuesType);
       final FieldInfo fi = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, omitNorms, storePayloads, indexOptions, docValuesType, normType, null);
       putInternal(fi);

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java?rev=1433646&r1=1433645&r2=1433646&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java Tue Jan 15 21:00:39 2013
@@ -406,12 +406,6 @@ public class FilterAtomicReader extends 
   }
 
   @Override
-  public DocValues docValues(String field) throws IOException {
-    ensureOpen();
-    return in.docValues(field);
-  }
-
-  @Override
   public NumericDocValues getNumericDocValues(String field) throws IOException {
     ensureOpen();
     return in.getNumericDocValues(field);
@@ -428,12 +422,6 @@ public class FilterAtomicReader extends 
     ensureOpen();
     return in.getSortedDocValues(field);
   }
-  
-  @Override
-  public DocValues normValues(String field) throws IOException {
-    ensureOpen();
-    return in.normValues(field);
-  }
 
   @Override
   public NumericDocValues simpleNormValues(String field) throws IOException {