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/02/08 04:26:21 UTC

svn commit: r1443834 [15/16] - in /lucene/dev/branches/branch_4x: ./ dev-tools/ lucene/ lucene/analysis/ lucene/analysis/icu/src/java/org/apache/lucene/collation/ lucene/analysis/icu/src/test/org/apache/lucene/collation/ lucene/backwards/ lucene/benchm...

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWNormsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWNormsConsumer.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWNormsConsumer.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWNormsConsumer.java Fri Feb  8 03:26:14 2013
@@ -18,30 +18,21 @@ package org.apache.lucene.codecs.lucene3
  */
 
 import java.io.IOException;
-import java.util.Arrays;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
-import org.apache.lucene.codecs.PerDocConsumer;
-import org.apache.lucene.index.AtomicReader;
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.DocValues.Source;
-import org.apache.lucene.index.DocValues.Type;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.IndexableField;
-import org.apache.lucene.index.MergeState;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 
 /**
  * Writes and Merges Lucene 3.x norms format
  * @lucene.experimental
  */
-class PreFlexRWNormsConsumer extends PerDocConsumer {
+class PreFlexRWNormsConsumer extends DocValuesConsumer {
   
   /** norms header placeholder */
   private static final byte[] NORMS_HEADER = new byte[]{'N','R','M',-1};
@@ -54,245 +45,49 @@ class PreFlexRWNormsConsumer extends Per
   @Deprecated
   private static final String SEPARATE_NORMS_EXTENSION = "s";
 
-  private final Directory directory;
-
-  private final String segment;
-
-  private final IOContext context;
-
-  private NormsWriter writer;
+  private final IndexOutput out;
+  private int lastFieldNumber = -1; // only for assert
   
-  public PreFlexRWNormsConsumer(Directory directory, String segment, IOContext context){
-    this.directory = directory;
-    this.segment = segment;
-    this.context = context;
+  public PreFlexRWNormsConsumer(Directory directory, String segment, IOContext context) throws IOException {
+    final String normsFileName = IndexFileNames.segmentFileName(segment, "", NORMS_EXTENSION);
+    boolean success = false;
+    IndexOutput output = null;
+    try {
+      output = directory.createOutput(normsFileName, context);
+      output.writeBytes(NORMS_HEADER, 0, NORMS_HEADER.length);
+      out = output;
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(output);
+      }
+    }
   }
 
   @Override
-  public void merge(MergeState mergeState) throws IOException {
-    getNormsWriter().merge(mergeState);
+  public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
+    assert field.number > lastFieldNumber : "writing norms fields out of order" + lastFieldNumber + " -> " + field.number;
+    for (Number n : values) {
+      if (n.longValue() < Byte.MIN_VALUE || n.longValue() > Byte.MAX_VALUE) {
+        throw new UnsupportedOperationException("3.x cannot index norms that won't fit in a byte, got: " + n.longValue());
+      }
+      out.writeByte(n.byteValue());
+    }
+    lastFieldNumber = field.number;
   }
 
   @Override
   public void close() throws IOException {
-    if (writer != null) {
-      writer.finish();
-    }
-  }
-  
-  @Override
-  protected boolean canMerge(FieldInfo info) {
-    return info.hasNorms();
+    IOUtils.close(out);
   }
 
   @Override
-  protected Type getDocValuesType(FieldInfo info) {
-    return info.getNormType();
+  public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
+    throw new AssertionError();
   }
 
   @Override
-  public DocValuesConsumer addValuesField(Type type, FieldInfo fieldInfo)
-      throws IOException {
-    if (type != Type.FIXED_INTS_8) {
-      throw new UnsupportedOperationException("Codec only supports single byte norm values. Type give: " + type);
-    }
-    return new Lucene3xNormsDocValuesConsumer(fieldInfo);
-  }
-
-  class Lucene3xNormsDocValuesConsumer extends DocValuesConsumer {
-    // Holds all docID/norm pairs we've seen
-    private int[] docIDs = new int[1];
-    private byte[] norms = new byte[1];
-    private int upto;
-    private final FieldInfo fi;
-    
-    Lucene3xNormsDocValuesConsumer(FieldInfo fieldInfo) {
-      fi = fieldInfo;
-    }
-
-    @Override
-    public void finish(int docCount) throws IOException {
-      final NormsWriter normsWriter = getNormsWriter();
-      boolean success = false;
-      try {
-        int uptoDoc = 0;
-        normsWriter.setNumTotalDocs(docCount);
-        if (upto > 0) {
-          normsWriter.startField(fi);
-          int docID = 0;
-          for (; docID < docCount; docID++) {
-            if (uptoDoc < upto && docIDs[uptoDoc] == docID) {
-              normsWriter.writeNorm(norms[uptoDoc]);
-              uptoDoc++;
-            } else {
-              normsWriter.writeNorm((byte) 0);
-            }
-          }
-          // we should have consumed every norm
-          assert uptoDoc == upto;
-  
-        } else {
-          // Fill entire field with default norm:
-          normsWriter.startField(fi);
-          for (; upto < docCount; upto++)
-            normsWriter.writeNorm((byte) 0);
-        }
-        success = true;
-      } finally {
-        if (!success) {
-          normsWriter.abort();
-        }
-      }
-    }
-    
-    @Override
-    public void add(int docID, IndexableField docValue) throws IOException {
-      add(docID, docValue.numericValue().longValue());
-    }
-    
-    protected void add(int docID, long value) {
-      if (docIDs.length <= upto) {
-        assert docIDs.length == upto;
-        docIDs = ArrayUtil.grow(docIDs, 1 + upto);
-      }
-      if (norms.length <= upto) {
-        assert norms.length == upto;
-        norms = ArrayUtil.grow(norms, 1 + upto);
-      }
-      norms[upto] = (byte) value;
-      
-      docIDs[upto] = docID;
-      upto++;
-    }
-
-    @Override
-    protected Type getType() {
-      return Type.FIXED_INTS_8;
-    }
-  
-    @Override
-    public int getValueSize() {
-      return 1;
-    }
-    
-  }
-  
-  public NormsWriter getNormsWriter() throws IOException {
-    if (writer == null) {
-      writer = new NormsWriter(directory, segment, context);
-    }
-    return writer;
-  }
-  
-  private static class NormsWriter {
-    
-    private final IndexOutput output;
-    private int normCount = 0;
-    private int numTotalDocs = 0;
-    
-    public NormsWriter(Directory directory, String segment, IOContext context) throws IOException {
-      final String normsFileName = IndexFileNames.segmentFileName(segment, "", NORMS_EXTENSION);
-      boolean success = false;
-      IndexOutput out = null;
-      try {
-        out = directory.createOutput(normsFileName, context);
-        output = out;
-        output.writeBytes(NORMS_HEADER, 0, NORMS_HEADER.length);
-        success = true;
-      } finally {
-        if (!success) {
-          IOUtils.closeWhileHandlingException(out);
-        }
-      }
-      
-    }
-    
-    
-    public void setNumTotalDocs(int numTotalDocs) {
-      assert this.numTotalDocs == 0 || numTotalDocs == this.numTotalDocs;
-      this.numTotalDocs = numTotalDocs;
-    }
-    
-    public void startField(FieldInfo info) throws IOException {
-      assert info.omitsNorms() == false;
-      normCount++;
-    }
-    
-    public void writeNorm(byte norm) throws IOException {
-      output.writeByte(norm);
-    }
-    
-    public void abort() throws IOException {
-      IOUtils.close(output);
-    }
-    
-    public void finish() throws IOException {
-      IOUtils.close(output);
-      
-      if (4+normCount*(long)numTotalDocs != output.getFilePointer()) {
-        throw new IOException(".nrm file size mismatch: expected=" + (4+normCount*(long)numTotalDocs) + " actual=" + output.getFilePointer());
-      }
-    }
-    // TODO: we can actually use the defaul DV merge here and drop this specific stuff entirely
-    /** we override merge and bulk-merge norms when there are no deletions */
-    public void merge(MergeState mergeState) throws IOException {
-      int numMergedDocs = 0;
-      for (FieldInfo fi : mergeState.fieldInfos) {
-        if (fi.hasNorms()) {
-          startField(fi);
-          int numMergedDocsForField = 0;
-          for (AtomicReader reader : mergeState.readers) {
-            final int maxDoc = reader.maxDoc();
-            byte[] normBuffer;
-            DocValues normValues = reader.normValues(fi.name);
-            if (normValues == null) {
-              // Can be null if this segment doesn't have
-              // any docs with this field
-              normBuffer = new byte[maxDoc];
-              Arrays.fill(normBuffer, (byte)0);
-            } else {
-              Source directSource = normValues.getDirectSource();
-              assert directSource.hasArray();
-              normBuffer = (byte[]) directSource.getArray();
-            }
-            if (reader.getLiveDocs() == null) {
-              //optimized case for segments without deleted docs
-              output.writeBytes(normBuffer, maxDoc);
-              numMergedDocsForField += maxDoc;
-            } else {
-              // this segment has deleted docs, so we have to
-              // check for every doc if it is deleted or not
-              final Bits liveDocs = reader.getLiveDocs();
-              for (int k = 0; k < maxDoc; k++) {
-                if (liveDocs.get(k)) {
-                  numMergedDocsForField++;
-                  output.writeByte(normBuffer[k]);
-                }
-              }
-            }
-            mergeState.checkAbort.work(maxDoc);
-          }
-          assert numMergedDocs == 0 || numMergedDocs == numMergedDocsForField;
-          numMergedDocs = numMergedDocsForField;
-        }
-      }
-      this.numTotalDocs = numMergedDocs;
-    }
-  }
-
-  @Override
-  public void abort() {
-    try {
-      try {
-        if (writer != null) {
-          writer.abort();
-        }
-      } finally {
-        directory.deleteFile(IndexFileNames.segmentFileName(segment, "",
-            NORMS_EXTENSION));
-      }
-    } catch (Throwable e) {
-      // ignore
-    }
+  public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
+    throw new AssertionError();
   }
 }

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWNormsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWNormsFormat.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWNormsFormat.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWNormsFormat.java Fri Feb  8 03:26:14 2013
@@ -19,8 +19,8 @@ package org.apache.lucene.codecs.lucene3
 
 import java.io.IOException;
 
-import org.apache.lucene.codecs.PerDocConsumer;
-import org.apache.lucene.index.PerDocWriteState;
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.index.SegmentWriteState;
 
 /**
  * @lucene.internal
@@ -29,7 +29,7 @@ import org.apache.lucene.index.PerDocWri
 class PreFlexRWNormsFormat extends Lucene3xNormsFormat {
 
   @Override
-  public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
+  public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
     return new PreFlexRWNormsConsumer(state.directory, state.segmentInfo.name, state.context);
   }
 }

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWPostingsFormat.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWPostingsFormat.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWPostingsFormat.java Fri Feb  8 03:26:14 2013
@@ -48,7 +48,7 @@ class PreFlexRWPostingsFormat extends Lu
     // Whenever IW opens readers, eg for merging, we have to
     // keep terms order in UTF16:
 
-    return new Lucene3xFields(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.termsIndexDivisor) {
+    return new Lucene3xFields(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.termsIndexDivisor) {
       @Override
       protected boolean sortTermsByUnicode() {
         // We carefully peek into stack track above us: if

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene41ords/Lucene41WithOrds.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene41ords/Lucene41WithOrds.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene41ords/Lucene41WithOrds.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene41ords/Lucene41WithOrds.java Fri Feb  8 03:26:14 2013
@@ -91,12 +91,12 @@ public final class Lucene41WithOrds exte
 
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
-    PostingsReaderBase postings = new Lucene41PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
+    PostingsReaderBase postings = new Lucene41PostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
     TermsIndexReaderBase indexReader;
 
     boolean success = false;
     try {
-      indexReader = new FixedGapTermsIndexReader(state.dir,
+      indexReader = new FixedGapTermsIndexReader(state.directory,
                                                  state.fieldInfos,
                                                  state.segmentInfo.name,
                                                  state.termsIndexDivisor,
@@ -112,7 +112,7 @@ public final class Lucene41WithOrds exte
     success = false;
     try {
       FieldsProducer ret = new BlockTermsReader(indexReader,
-                                                state.dir,
+                                                state.directory,
                                                 state.fieldInfos,
                                                 state.segmentInfo,
                                                 postings,

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java Fri Feb  8 03:26:14 2013
@@ -157,7 +157,7 @@ public final class MockFixedIntBlockPost
 
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
-    PostingsReaderBase postingsReader = new SepPostingsReader(state.dir,
+    PostingsReaderBase postingsReader = new SepPostingsReader(state.directory,
                                                               state.fieldInfos,
                                                               state.segmentInfo,
                                                               state.context,
@@ -166,7 +166,7 @@ public final class MockFixedIntBlockPost
     TermsIndexReaderBase indexReader;
     boolean success = false;
     try {
-      indexReader = new FixedGapTermsIndexReader(state.dir,
+      indexReader = new FixedGapTermsIndexReader(state.directory,
                                                        state.fieldInfos,
                                                        state.segmentInfo.name,
                                                        state.termsIndexDivisor,
@@ -182,7 +182,7 @@ public final class MockFixedIntBlockPost
     success = false;
     try {
       FieldsProducer ret = new BlockTermsReader(indexReader,
-                                                state.dir,
+                                                state.directory,
                                                 state.fieldInfos,
                                                 state.segmentInfo,
                                                 postingsReader,

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java Fri Feb  8 03:26:14 2013
@@ -182,7 +182,7 @@ public final class MockVariableIntBlockP
 
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
-    PostingsReaderBase postingsReader = new SepPostingsReader(state.dir,
+    PostingsReaderBase postingsReader = new SepPostingsReader(state.directory,
                                                               state.fieldInfos,
                                                               state.segmentInfo,
                                                               state.context,
@@ -191,7 +191,7 @@ public final class MockVariableIntBlockP
     TermsIndexReaderBase indexReader;
     boolean success = false;
     try {
-      indexReader = new FixedGapTermsIndexReader(state.dir,
+      indexReader = new FixedGapTermsIndexReader(state.directory,
                                                        state.fieldInfos,
                                                        state.segmentInfo.name,
                                                        state.termsIndexDivisor,
@@ -207,7 +207,7 @@ public final class MockVariableIntBlockP
     success = false;
     try {
       FieldsProducer ret = new BlockTermsReader(indexReader,
-                                                state.dir,
+                                                state.directory,
                                                 state.fieldInfos,
                                                 state.segmentInfo,
                                                 postingsReader,

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java Fri Feb  8 03:26:14 2013
@@ -288,7 +288,7 @@ public final class MockRandomPostingsFor
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
 
     final String seedFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, SEED_EXT);
-    final IndexInput in = state.dir.openInput(seedFileName, state.context);
+    final IndexInput in = state.directory.openInput(seedFileName, state.context);
     final long seed = in.readLong();
     if (LuceneTestCase.VERBOSE) {
       System.out.println("MockRandomCodec: reading from seg=" + state.segmentInfo.name + " formatID=" + state.segmentSuffix + " seed=" + seed);
@@ -308,13 +308,13 @@ public final class MockRandomPostingsFor
       if (LuceneTestCase.VERBOSE) {
         System.out.println("MockRandomCodec: reading Sep postings");
       }
-      postingsReader = new SepPostingsReader(state.dir, state.fieldInfos, state.segmentInfo,
+      postingsReader = new SepPostingsReader(state.directory, state.fieldInfos, state.segmentInfo,
                                              state.context, new MockIntStreamFactory(random), state.segmentSuffix);
     } else {
       if (LuceneTestCase.VERBOSE) {
         System.out.println("MockRandomCodec: reading Standard postings");
       }
-      postingsReader = new Lucene41PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
+      postingsReader = new Lucene41PostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
     }
 
     if (random.nextBoolean()) {
@@ -335,7 +335,7 @@ public final class MockRandomPostingsFor
 
       boolean success = false;
       try {
-        fields = new BlockTreeTermsReader(state.dir,
+        fields = new BlockTreeTermsReader(state.directory,
                                           state.fieldInfos,
                                           state.segmentInfo,
                                           postingsReader,
@@ -369,7 +369,7 @@ public final class MockRandomPostingsFor
           if (LuceneTestCase.VERBOSE) {
             System.out.println("MockRandomCodec: fixed-gap terms index (divisor=" + state.termsIndexDivisor + ")");
           }
-          indexReader = new FixedGapTermsIndexReader(state.dir,
+          indexReader = new FixedGapTermsIndexReader(state.directory,
                                                      state.fieldInfos,
                                                      state.segmentInfo.name,
                                                      state.termsIndexDivisor,
@@ -385,7 +385,7 @@ public final class MockRandomPostingsFor
           if (LuceneTestCase.VERBOSE) {
             System.out.println("MockRandomCodec: variable-gap terms index (divisor=" + state.termsIndexDivisor + ")");
           }
-          indexReader = new VariableGapTermsIndexReader(state.dir,
+          indexReader = new VariableGapTermsIndexReader(state.directory,
                                                         state.fieldInfos,
                                                         state.segmentInfo.name,
                                                         state.termsIndexDivisor,
@@ -405,7 +405,7 @@ public final class MockRandomPostingsFor
       success = false;
       try {
         fields = new BlockTermsReader(indexReader,
-                                      state.dir,
+                                      state.directory,
                                       state.fieldInfos,
                                       state.segmentInfo,
                                       postingsReader,

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSepPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSepPostingsFormat.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSepPostingsFormat.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSepPostingsFormat.java Fri Feb  8 03:26:14 2013
@@ -83,13 +83,13 @@ public final class MockSepPostingsFormat
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
 
-    PostingsReaderBase postingsReader = new SepPostingsReader(state.dir, state.fieldInfos, state.segmentInfo,
+    PostingsReaderBase postingsReader = new SepPostingsReader(state.directory, state.fieldInfos, state.segmentInfo,
         state.context, new MockSingleIntFactory(), state.segmentSuffix);
 
     TermsIndexReaderBase indexReader;
     boolean success = false;
     try {
-      indexReader = new FixedGapTermsIndexReader(state.dir,
+      indexReader = new FixedGapTermsIndexReader(state.directory,
                                                        state.fieldInfos,
                                                        state.segmentInfo.name,
                                                        state.termsIndexDivisor,
@@ -105,7 +105,7 @@ public final class MockSepPostingsFormat
     success = false;
     try {
       FieldsProducer ret = new BlockTermsReader(indexReader,
-                                                state.dir,
+                                                state.directory,
                                                 state.fieldInfos,
                                                 state.segmentInfo,
                                                 postingsReader,

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java Fri Feb  8 03:26:14 2013
@@ -77,11 +77,11 @@ public final class NestedPulsingPostings
     PostingsReaderBase pulsingReader = null;
     boolean success = false;
     try {
-      docsReader = new Lucene41PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
+      docsReader = new Lucene41PostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
       pulsingReaderInner = new PulsingPostingsReader(docsReader);
       pulsingReader = new PulsingPostingsReader(pulsingReaderInner);
       FieldsProducer ret = new BlockTreeTermsReader(
-                                                    state.dir, state.fieldInfos, state.segmentInfo,
+                                                    state.directory, state.fieldInfos, state.segmentInfo,
                                                     pulsingReader,
                                                     state.context,
                                                     state.segmentSuffix,

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java Fri Feb  8 03:26:14 2013
@@ -555,7 +555,7 @@ public final class RAMOnlyPostingsFormat
 
     // Load our ID:
     final String idFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name, readState.segmentSuffix, ID_EXTENSION);
-    IndexInput in = readState.dir.openInput(idFileName, readState.context);
+    IndexInput in = readState.directory.openInput(idFileName, readState.context);
     boolean success = false;
     final int id;
     try {

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/AssertingAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/AssertingAtomicReader.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/AssertingAtomicReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/AssertingAtomicReader.java Fri Feb  8 03:26:14 2013
@@ -86,6 +86,7 @@ public class AssertingAtomicReader exten
     public TermsEnum intersect(CompiledAutomaton automaton, BytesRef bytes) throws IOException {
       TermsEnum termsEnum = super.intersect(automaton, bytes);
       assert termsEnum != null;
+      assert bytes == null || bytes.isValid();
       return new AssertingTermsEnum(termsEnum);
     }
 
@@ -145,6 +146,7 @@ public class AssertingAtomicReader exten
       if (result == null) {
         state = State.UNPOSITIONED;
       } else {
+        assert result.isValid();
         state = State.POSITIONED;
       }
       return result;
@@ -171,7 +173,9 @@ public class AssertingAtomicReader exten
     @Override
     public BytesRef term() throws IOException {
       assert state == State.POSITIONED : "term() called on unpositioned TermsEnum";
-      return super.term();
+      BytesRef ret = super.term();
+      assert ret == null || ret.isValid();
+      return ret;
     }
 
     @Override
@@ -182,6 +186,7 @@ public class AssertingAtomicReader exten
 
     @Override
     public SeekStatus seekCeil(BytesRef term, boolean useCache) throws IOException {
+      assert term.isValid();
       SeekStatus result = super.seekCeil(term, useCache);
       if (result == SeekStatus.END) {
         state = State.UNPOSITIONED;
@@ -193,6 +198,7 @@ public class AssertingAtomicReader exten
 
     @Override
     public boolean seekExact(BytesRef text, boolean useCache) throws IOException {
+      assert text.isValid();
       if (super.seekExact(text, useCache)) {
         state = State.POSITIONED;
         return true;
@@ -210,6 +216,7 @@ public class AssertingAtomicReader exten
 
     @Override
     public void seekExact(BytesRef term, TermState state) throws IOException {
+      assert term.isValid();
       super.seekExact(term, state);
       this.state = State.POSITIONED;
     }
@@ -352,10 +359,156 @@ public class AssertingAtomicReader exten
       assert state != DocsEnumState.FINISHED : "getPayload() called after NO_MORE_DOCS";
       assert positionCount > 0 : "getPayload() called before nextPosition()!";
       BytesRef payload = super.getPayload();
-      assert payload == null || payload.length > 0 : "getPayload() returned payload with invalid length!";
+      assert payload == null || payload.isValid() && payload.length > 0 : "getPayload() returned payload with invalid length!";
       return payload;
     }
   }
+  
+  /** Wraps a NumericDocValues but with additional asserts */
+  public static class AssertingNumericDocValues extends NumericDocValues {
+    private final NumericDocValues in;
+    private final int maxDoc;
+    
+    public AssertingNumericDocValues(NumericDocValues in, int maxDoc) {
+      this.in = in;
+      this.maxDoc = maxDoc;
+    }
+
+    @Override
+    public long get(int docID) {
+      assert docID >= 0 && docID < maxDoc;
+      return in.get(docID);
+    }    
+  }
+  
+  /** Wraps a BinaryDocValues but with additional asserts */
+  public static class AssertingBinaryDocValues extends BinaryDocValues {
+    private final BinaryDocValues in;
+    private final int maxDoc;
+    
+    public AssertingBinaryDocValues(BinaryDocValues in, int maxDoc) {
+      this.in = in;
+      this.maxDoc = maxDoc;
+    }
+
+    @Override
+    public void get(int docID, BytesRef result) {
+      assert docID >= 0 && docID < maxDoc;
+      assert result.isValid();
+      in.get(docID, result);
+      assert result.isValid();
+    }
+  }
+  
+  /** Wraps a SortedDocValues but with additional asserts */
+  public static class AssertingSortedDocValues extends SortedDocValues {
+    private final SortedDocValues in;
+    private final int maxDoc;
+    private final int valueCount;
+    
+    public AssertingSortedDocValues(SortedDocValues in, int maxDoc) {
+      this.in = in;
+      this.maxDoc = maxDoc;
+      this.valueCount = in.getValueCount();
+      assert valueCount >= 1 && valueCount <= maxDoc;
+    }
+
+    @Override
+    public int getOrd(int docID) {
+      assert docID >= 0 && docID < maxDoc;
+      int ord = in.getOrd(docID);
+      assert ord >= 0 && ord < valueCount;
+      return ord;
+    }
+
+    @Override
+    public void lookupOrd(int ord, BytesRef result) {
+      assert ord >= 0 && ord < valueCount;
+      assert result.isValid();
+      in.lookupOrd(ord, result);
+      assert result.isValid();
+    }
+
+    @Override
+    public int getValueCount() {
+      int valueCount = in.getValueCount();
+      assert valueCount == this.valueCount; // should not change
+      return valueCount;
+    }
+
+    @Override
+    public void get(int docID, BytesRef result) {
+      assert docID >= 0 && docID < maxDoc;
+      assert result.isValid();
+      in.get(docID, result);
+      assert result.isValid();
+    }
+
+    @Override
+    public int lookupTerm(BytesRef key) {
+      assert key.isValid();
+      int result = in.lookupTerm(key);
+      assert result < valueCount;
+      assert key.isValid();
+      return result;
+    }
+  }
+
+  @Override
+  public NumericDocValues getNumericDocValues(String field) throws IOException {
+    NumericDocValues dv = super.getNumericDocValues(field);
+    FieldInfo fi = getFieldInfos().fieldInfo(field);
+    if (dv != null) {
+      assert fi != null;
+      assert fi.getDocValuesType() == FieldInfo.DocValuesType.NUMERIC;
+      return new AssertingNumericDocValues(dv, maxDoc());
+    } else {
+      assert fi == null || fi.getDocValuesType() != FieldInfo.DocValuesType.NUMERIC;
+      return null;
+    }
+  }
+
+  @Override
+  public BinaryDocValues getBinaryDocValues(String field) throws IOException {
+    BinaryDocValues dv = super.getBinaryDocValues(field);
+    FieldInfo fi = getFieldInfos().fieldInfo(field);
+    if (dv != null) {
+      assert fi != null;
+      assert fi.getDocValuesType() == FieldInfo.DocValuesType.BINARY;
+      return new AssertingBinaryDocValues(dv, maxDoc());
+    } else {
+      assert fi == null || fi.getDocValuesType() != FieldInfo.DocValuesType.BINARY;
+      return null;
+    }
+  }
+
+  @Override
+  public SortedDocValues getSortedDocValues(String field) throws IOException {
+    SortedDocValues dv = super.getSortedDocValues(field);
+    FieldInfo fi = getFieldInfos().fieldInfo(field);
+    if (dv != null) {
+      assert fi != null;
+      assert fi.getDocValuesType() == FieldInfo.DocValuesType.SORTED;
+      return new AssertingSortedDocValues(dv, maxDoc());
+    } else {
+      assert fi == null || fi.getDocValuesType() != FieldInfo.DocValuesType.SORTED;
+      return null;
+    }
+  }
+
+  @Override
+  public NumericDocValues getNormValues(String field) throws IOException {
+    NumericDocValues dv = super.getNormValues(field);
+    FieldInfo fi = getFieldInfos().fieldInfo(field);
+    if (dv != null) {
+      assert fi != null;
+      assert fi.hasNorms();
+      return new AssertingNumericDocValues(dv, maxDoc());
+    } else {
+      assert fi == null || fi.hasNorms() == false;
+      return null;
+    }
+  }
 
   // this is the same hack as FCInvisible
   @Override

Copied: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java (from r1443717, lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java?p2=lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java&p1=lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java&r1=1443717&r2=1443834&rev=1443834&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java Fri Feb  8 03:26:14 2013
@@ -99,7 +99,7 @@ public abstract class BaseDocValuesForma
     assertEquals(1, hits.totalHits);
     // Iterate through the results:
     for (int i = 0; i < hits.scoreDocs.length; i++) {
-      StoredDocument hitDoc = isearcher.doc(hits.scoreDocs[i].doc);
+      Document hitDoc = isearcher.doc(hits.scoreDocs[i].doc);
       assertEquals(text, hitDoc.get("fieldname"));
       assert ireader.leaves().size() == 1;
       NumericDocValues dv = ireader.leaves().get(0).reader().getNumericDocValues("dv");
@@ -131,7 +131,7 @@ public abstract class BaseDocValuesForma
     assertEquals(1, hits.totalHits);
     // Iterate through the results:
     for (int i = 0; i < hits.scoreDocs.length; i++) {
-      StoredDocument hitDoc = isearcher.doc(hits.scoreDocs[i].doc);
+      Document hitDoc = isearcher.doc(hits.scoreDocs[i].doc);
       assertEquals(text, hitDoc.get("fieldname"));
       assert ireader.leaves().size() == 1;
       NumericDocValues dv = ireader.leaves().get(0).reader().getNumericDocValues("dv");
@@ -164,7 +164,7 @@ public abstract class BaseDocValuesForma
     assertEquals(1, hits.totalHits);
     // Iterate through the results:
     for (int i = 0; i < hits.scoreDocs.length; i++) {
-      StoredDocument hitDoc = isearcher.doc(hits.scoreDocs[i].doc);
+      Document hitDoc = isearcher.doc(hits.scoreDocs[i].doc);
       assertEquals(text, hitDoc.get("fieldname"));
       assert ireader.leaves().size() == 1;
       NumericDocValues dv = ireader.leaves().get(0).reader().getNumericDocValues("dv1");
@@ -200,7 +200,7 @@ public abstract class BaseDocValuesForma
     BytesRef scratch = new BytesRef();
     // Iterate through the results:
     for (int i = 0; i < hits.scoreDocs.length; i++) {
-      StoredDocument hitDoc = isearcher.doc(hits.scoreDocs[i].doc);
+      Document hitDoc = isearcher.doc(hits.scoreDocs[i].doc);
       assertEquals(text, hitDoc.get("fieldname"));
       assert ireader.leaves().size() == 1;
       NumericDocValues dv = ireader.leaves().get(0).reader().getNumericDocValues("dv1");
@@ -238,7 +238,7 @@ public abstract class BaseDocValuesForma
     BytesRef scratch = new BytesRef();
     // Iterate through the results:
     for (int i = 0; i < hits.scoreDocs.length; i++) {
-      StoredDocument hitDoc = isearcher.doc(hits.scoreDocs[i].doc);
+      Document hitDoc = isearcher.doc(hits.scoreDocs[i].doc);
       assertEquals(text, hitDoc.get("fieldname"));
       assert ireader.leaves().size() == 1;
       SortedDocValues dv = ireader.leaves().get(0).reader().getSortedDocValues("dv1");
@@ -280,7 +280,7 @@ public abstract class BaseDocValuesForma
     BytesRef scratch = new BytesRef();
     // Iterate through the results:
     for (int i = 0; i < hits.scoreDocs.length; i++) {
-      StoredDocument hitDoc = isearcher.doc(hits.scoreDocs[i].doc);
+      Document hitDoc = isearcher.doc(hits.scoreDocs[i].doc);
       assertEquals(text, hitDoc.get("fieldname"));
       assert ireader.leaves().size() == 1;
       SortedDocValues dv = ireader.leaves().get(0).reader().getSortedDocValues("dv2");
@@ -349,7 +349,7 @@ public abstract class BaseDocValuesForma
     assert ireader.leaves().size() == 1;
     NumericDocValues dv = ireader.leaves().get(0).reader().getNumericDocValues("dv");
     for(int i=0;i<2;i++) {
-      StoredDocument doc2 = ireader.leaves().get(0).reader().document(i);
+      Document doc2 = ireader.leaves().get(0).reader().document(i);
       long expected;
       if (doc2.get("id").equals("0")) {
         expected = -10;
@@ -442,7 +442,7 @@ public abstract class BaseDocValuesForma
     BytesRef scratch = new BytesRef();
     // Iterate through the results:
     for (int i = 0; i < hits.scoreDocs.length; i++) {
-      StoredDocument hitDoc = isearcher.doc(hits.scoreDocs[i].doc);
+      Document hitDoc = isearcher.doc(hits.scoreDocs[i].doc);
       assertEquals(text, hitDoc.get("fieldname"));
       assert ireader.leaves().size() == 1;
       BinaryDocValues dv = ireader.leaves().get(0).reader().getBinaryDocValues("dv");
@@ -479,7 +479,7 @@ public abstract class BaseDocValuesForma
     BinaryDocValues dv = ireader.leaves().get(0).reader().getBinaryDocValues("dv");
     BytesRef scratch = new BytesRef();
     for(int i=0;i<2;i++) {
-      StoredDocument doc2 = ireader.leaves().get(0).reader().document(i);
+      Document doc2 = ireader.leaves().get(0).reader().document(i);
       String expected;
       if (doc2.get("id").equals("0")) {
         expected = "hello world 1";
@@ -519,7 +519,7 @@ public abstract class BaseDocValuesForma
     BytesRef scratch = new BytesRef();
     // Iterate through the results:
     for (int i = 0; i < hits.scoreDocs.length; i++) {
-      StoredDocument hitDoc = isearcher.doc(hits.scoreDocs[i].doc);
+      Document hitDoc = isearcher.doc(hits.scoreDocs[i].doc);
       assertEquals(text, hitDoc.get("fieldname"));
       assert ireader.leaves().size() == 1;
       SortedDocValues dv = ireader.leaves().get(0).reader().getSortedDocValues("dv");
@@ -628,7 +628,7 @@ public abstract class BaseDocValuesForma
     dv.lookupOrd(1, scratch);
     assertEquals(new BytesRef("hello world 2"), scratch);
     for(int i=0;i<2;i++) {
-      StoredDocument doc2 = ireader.leaves().get(0).reader().document(i);
+      Document doc2 = ireader.leaves().get(0).reader().document(i);
       String expected;
       if (doc2.get("id").equals("0")) {
         expected = "hello world 1";

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java Fri Feb  8 03:26:14 2013
@@ -38,6 +38,7 @@ import org.apache.lucene.codecs.FieldsPr
 import org.apache.lucene.codecs.PostingsConsumer;
 import org.apache.lucene.codecs.TermStats;
 import org.apache.lucene.codecs.TermsConsumer;
+import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FlushInfo;
@@ -345,7 +346,7 @@ public abstract class BasePostingsFormat
 
       fieldInfoArray[fieldUpto] = new FieldInfo(field, true, fieldUpto, false, false, true,
                                                 IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS,
-                                                null, DocValues.Type.FIXED_INTS_8, null);
+                                                null, DocValuesType.NUMERIC, null);
       fieldUpto++;
 
       Map<BytesRef,Long> postings = new TreeMap<BytesRef,Long>();
@@ -470,7 +471,7 @@ public abstract class BasePostingsFormat
                                                    doPayloads,
                                                    indexOptions,
                                                    null,
-                                                   DocValues.Type.FIXED_INTS_8,
+                                                   DocValuesType.NUMERIC,
                                                    null);
     }
 

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java Fri Feb  8 03:26:14 2013
@@ -113,15 +113,27 @@ public final class FieldFilterAtomicRead
     final Fields f = super.fields();
     return (f == null) ? null : new FieldFilterFields(f);
   }
+  
+  
+
+  @Override
+  public NumericDocValues getNumericDocValues(String field) throws IOException {
+    return hasField(field) ? super.getNumericDocValues(field) : null;
+  }
+
+  @Override
+  public BinaryDocValues getBinaryDocValues(String field) throws IOException {
+    return hasField(field) ? super.getBinaryDocValues(field) : null;
+  }
 
   @Override
-  public DocValues docValues(String field) throws IOException {
-    return hasField(field) ? super.docValues(field) : null;
+  public SortedDocValues getSortedDocValues(String field) throws IOException {
+    return hasField(field) ? super.getSortedDocValues(field) : null;
   }
 
   @Override
-  public DocValues normValues(String field) throws IOException {
-    return hasField(field) ? super.normValues(field) : null;
+  public NumericDocValues getNormValues(String field) throws IOException {
+    return hasField(field) ? super.getNormValues(field) : null;
   }
 
   @Override

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java Fri Feb  8 03:26:14 2013
@@ -28,11 +28,16 @@ import java.util.Random;
 import java.util.Set;
 
 import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.asserting.AssertingDocValuesFormat;
 import org.apache.lucene.codecs.asserting.AssertingPostingsFormat;
 import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
 import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
 import org.apache.lucene.codecs.lucene42.Lucene42Codec;
+import org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat;
 import org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings;
+import org.apache.lucene.codecs.cheapbastard.CheapBastardDocValuesFormat;
+import org.apache.lucene.codecs.diskdv.DiskDocValuesFormat;
 import org.apache.lucene.codecs.memory.DirectPostingsFormat;
 import org.apache.lucene.codecs.memory.MemoryPostingsFormat;
 import org.apache.lucene.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
@@ -42,6 +47,7 @@ import org.apache.lucene.codecs.mocksep.
 import org.apache.lucene.codecs.nestedpulsing.NestedPulsingPostingsFormat;
 import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
 import org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat;
+import org.apache.lucene.codecs.simpletext.SimpleTextDocValuesFormat;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util._TestUtil;
 
@@ -58,14 +64,21 @@ public class RandomCodec extends Lucene4
   /** Shuffled list of postings formats to use for new mappings */
   private List<PostingsFormat> formats = new ArrayList<PostingsFormat>();
   
+  /** Shuffled list of docvalues formats to use for new mappings */
+  private List<DocValuesFormat> dvFormats = new ArrayList<DocValuesFormat>();
+  
   /** unique set of format names this codec knows about */
   public Set<String> formatNames = new HashSet<String>();
+  
+  /** unique set of docvalues format names this codec knows about */
+  public Set<String> dvFormatNames = new HashSet<String>();
 
   /** memorized field->postingsformat mappings */
   // note: we have to sync this map even though its just for debugging/toString, 
   // otherwise DWPT's .toString() calls that iterate over the map can 
   // cause concurrentmodificationexception if indexwriter's infostream is on
   private Map<String,PostingsFormat> previousMappings = Collections.synchronizedMap(new HashMap<String,PostingsFormat>());
+  private Map<String,DocValuesFormat> previousDVMappings = Collections.synchronizedMap(new HashMap<String,DocValuesFormat>());
   private final int perFieldSeed;
 
   @Override
@@ -84,6 +97,22 @@ public class RandomCodec extends Lucene4
     return codec;
   }
 
+  @Override
+  public DocValuesFormat getDocValuesFormatForField(String name) {
+    DocValuesFormat codec = previousDVMappings.get(name);
+    if (codec == null) {
+      codec = dvFormats.get(Math.abs(perFieldSeed ^ name.hashCode()) % dvFormats.size());
+      if (codec instanceof SimpleTextDocValuesFormat && perFieldSeed % 5 != 0) {
+        // make simpletext rarer, choose again
+        codec = dvFormats.get(Math.abs(perFieldSeed ^ name.toUpperCase(Locale.ROOT).hashCode()) % dvFormats.size());
+      }
+      previousDVMappings.put(name, codec);
+      // Safety:
+      assert previousDVMappings.size() < 10000: "test went insane";
+    }
+    return codec;
+  }
+
   public RandomCodec(Random random, Set<String> avoidCodecs) {
     this.perFieldSeed = random.nextInt();
     // TODO: make it possible to specify min/max iterms per
@@ -113,11 +142,20 @@ public class RandomCodec extends Lucene4
         new AssertingPostingsFormat(),
         new MemoryPostingsFormat(true, random.nextFloat()),
         new MemoryPostingsFormat(false, random.nextFloat()));
+    
+    addDocValues(avoidCodecs,
+        new Lucene42DocValuesFormat(),
+        new DiskDocValuesFormat(),
+        new SimpleTextDocValuesFormat(),
+        new AssertingDocValuesFormat(),
+        new CheapBastardDocValuesFormat());
 
     Collections.shuffle(formats, random);
+    Collections.shuffle(dvFormats, random);
 
     // Avoid too many open files:
     formats = formats.subList(0, 4);
+    dvFormats = dvFormats.subList(0, 4);
   }
 
   public RandomCodec(Random random) {
@@ -132,9 +170,18 @@ public class RandomCodec extends Lucene4
       }
     }
   }
+  
+  private final void addDocValues(Set<String> avoidCodecs, DocValuesFormat... docvalues) {
+    for (DocValuesFormat d : docvalues) {
+      if (!avoidCodecs.contains(d.getName())) {
+        dvFormats.add(d);
+        dvFormatNames.add(d.getName());
+      }
+    }
+  }
 
   @Override
   public String toString() {
-    return super.toString() + ": " + previousMappings.toString();
+    return super.toString() + ": " + previousMappings.toString() + ", docValues:" + previousDVMappings.toString();
   }
 }

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java Fri Feb  8 03:26:14 2013
@@ -25,18 +25,12 @@ import java.util.Random;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.document.ByteDocValuesField; 
-import org.apache.lucene.document.DerefBytesDocValuesField; 
+import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.DoubleDocValuesField; 
 import org.apache.lucene.document.Field;
-import org.apache.lucene.document.FloatDocValuesField; 
-import org.apache.lucene.document.IntDocValuesField; 
-import org.apache.lucene.document.LongDocValuesField; 
-import org.apache.lucene.document.PackedLongDocValuesField; 
-import org.apache.lucene.document.ShortDocValuesField; 
-import org.apache.lucene.document.SortedBytesDocValuesField; 
-import org.apache.lucene.document.StraightBytesDocValuesField; 
+import org.apache.lucene.document.NumericDocValuesField; 
+import org.apache.lucene.document.SortedDocValuesField; 
+import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.index.IndexWriter; // javadoc
 import org.apache.lucene.search.Query;
 import org.apache.lucene.store.Directory;
@@ -59,9 +53,6 @@ public class RandomIndexWriter implement
   int flushAt;
   private double flushAtFactor = 1.0;
   private boolean getReaderCalled;
-  private final int fixedBytesLength;
-  private final long docValuesFieldPrefix;
-  private volatile boolean doDocValues;
   private final Codec codec; // sugar
 
   // Randomly calls Thread.yield so we mixup thread scheduling
@@ -109,49 +100,12 @@ public class RandomIndexWriter implement
       System.out.println("RIW dir=" + dir + " config=" + w.getConfig());
       System.out.println("codec default=" + codec.getName());
     }
-    /* TODO: find some way to make this random...
-     * This must be fixed across all fixed bytes 
-     * fields in one index. so if you open another writer
-     * this might change if I use r.nextInt(x)
-     * maybe we can peek at the existing files here? 
-     */
-    fixedBytesLength = 17; 
-
-    // NOTE: this means up to 13 * 5 unique fields (we have
-    // 13 different DV types):
-    docValuesFieldPrefix = r.nextInt(5);
-    switchDoDocValues();
 
     // Make sure we sometimes test indices that don't get
     // any forced merges:
     doRandomForceMerge = r.nextBoolean();
   } 
   
-  private boolean addDocValuesFields = true;
-  
-  /**
-   * set to false if you don't want RandomIndexWriter
-   * adding docvalues fields.
-   */
-  public void setAddDocValuesFields(boolean v) {
-    addDocValuesFields = v;
-    switchDoDocValues();
-  }
-
-  private void switchDoDocValues() {
-    if (addDocValuesFields == false) {
-      doDocValues = false;
-      return;
-    }
-    // randomly enable / disable docValues 
-    doDocValues = LuceneTestCase.rarely(r);
-    if (LuceneTestCase.VERBOSE) {
-      if (doDocValues) {
-        System.out.println("NOTE: RIW: turning on random DocValues fields");
-      }
-    }
-  }
-  
   /**
    * Adds a Document.
    * @see IndexWriter#addDocument(Iterable)
@@ -161,9 +115,6 @@ public class RandomIndexWriter implement
   }
 
   public <T extends IndexableField> void addDocument(final Iterable<T> doc, Analyzer a) throws IOException {
-    if (doDocValues && doc instanceof Document) {
-      randomPerDocFieldValues((Document) doc);
-    }
     if (r.nextInt(5) == 3) {
       // TODO: maybe, we should simply buffer up added docs
       // (but we need to clone them), and only when
@@ -204,75 +155,6 @@ public class RandomIndexWriter implement
     maybeCommit();
   }
 
-  private BytesRef getFixedRandomBytes() {
-    final String randomUnicodeString = _TestUtil.randomFixedByteLengthUnicodeString(r, fixedBytesLength);
-    BytesRef fixedRef = new BytesRef(randomUnicodeString);
-    if (fixedRef.length > fixedBytesLength) {
-      fixedRef = new BytesRef(fixedRef.bytes, 0, fixedBytesLength);
-    } else {
-      fixedRef.grow(fixedBytesLength);
-      fixedRef.length = fixedBytesLength;
-    }
-    return fixedRef;
-  }
-  
-  private void randomPerDocFieldValues(Document doc) {
-    
-    DocValues.Type[] values = DocValues.Type.values();
-    DocValues.Type type = values[r.nextInt(values.length)];
-    String name = "random_" + type.name() + "" + docValuesFieldPrefix;
-    if ("Lucene3x".equals(codec.getName()) || doc.getField(name) != null) {
-      return;
-    }
-    final Field f;
-    switch (type) {
-    case BYTES_FIXED_DEREF:
-      f = new DerefBytesDocValuesField(name, getFixedRandomBytes(), true);
-      break;
-    case BYTES_VAR_DEREF:
-      f = new DerefBytesDocValuesField(name, new BytesRef(_TestUtil.randomUnicodeString(r, 20)), false);
-      break;
-    case BYTES_FIXED_STRAIGHT:
-      f = new StraightBytesDocValuesField(name, getFixedRandomBytes(), true);
-      break;
-    case BYTES_VAR_STRAIGHT:
-      f = new StraightBytesDocValuesField(name, new BytesRef(_TestUtil.randomUnicodeString(r, 20)), false);
-      break;
-    case BYTES_FIXED_SORTED:
-      f = new SortedBytesDocValuesField(name, getFixedRandomBytes(), true);
-      break;
-    case BYTES_VAR_SORTED:
-      f = new SortedBytesDocValuesField(name, new BytesRef(_TestUtil.randomUnicodeString(r, 20)), false);
-      break;
-    case FLOAT_32:
-      f = new FloatDocValuesField(name, r.nextFloat());
-      break;
-    case FLOAT_64:
-      f = new DoubleDocValuesField(name, r.nextDouble());
-      break;
-    case VAR_INTS:
-      f = new PackedLongDocValuesField(name, r.nextLong());
-      break;
-    case FIXED_INTS_16:
-      // TODO: we should test negatives too?
-      f = new ShortDocValuesField(name, (short) r.nextInt(Short.MAX_VALUE));
-      break;
-    case FIXED_INTS_32:
-      f = new IntDocValuesField(name, r.nextInt());
-      break;
-    case FIXED_INTS_64:
-      f = new LongDocValuesField(name, r.nextLong());
-      break;
-    case FIXED_INTS_8:  
-      // TODO: we should test negatives too?
-      f = new ByteDocValuesField(name, (byte) r.nextInt(128));
-      break;
-    default:
-      throw new IllegalArgumentException("no such type: " + type);
-    }
-    doc.add(f);
-  }
-
   private void maybeCommit() throws IOException {
     if (docCount++ == flushAt) {
       if (LuceneTestCase.VERBOSE) {
@@ -284,7 +166,6 @@ public class RandomIndexWriter implement
         // gradually but exponentially increase time b/w flushes
         flushAtFactor *= 1.05;
       }
-      switchDoDocValues();
     }
   }
   
@@ -303,9 +184,6 @@ public class RandomIndexWriter implement
    * @see IndexWriter#updateDocument(Term, Iterable)
    */
   public <T extends IndexableField> void updateDocument(Term t, final Iterable<T> doc) throws IOException {
-    if (doDocValues) {
-      randomPerDocFieldValues((Document) doc);
-    }
     if (r.nextInt(5) == 3) {
       w.updateDocuments(t, new Iterable<Iterable<T>>() {
 
@@ -359,7 +237,6 @@ public class RandomIndexWriter implement
   
   public void commit() throws IOException {
     w.commit();
-    switchDoDocValues();
   }
   
   public int numDocs() {
@@ -416,7 +293,6 @@ public class RandomIndexWriter implement
         assert !doRandomForceMergeAssert || w.getSegmentCount() <= limit: "limit=" + limit + " actual=" + w.getSegmentCount();
       }
     }
-    switchDoDocValues();
   }
 
   public DirectoryReader getReader(boolean applyDeletions) throws IOException {
@@ -440,7 +316,6 @@ public class RandomIndexWriter implement
         System.out.println("RIW.getReader: open new reader");
       }
       w.commit();
-      switchDoDocValues();
       if (r.nextBoolean()) {
         return DirectoryReader.open(w.getDirectory(), _TestUtil.nextInt(r, 1, 10));
       } else {

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java Fri Feb  8 03:26:14 2013
@@ -638,7 +638,6 @@ public abstract class ThreadedIndexingAn
     int hitCount = s.search(q, null, 10, new Sort(new SortField("title", SortField.Type.STRING))).totalHits;
     if (defaultCodecSupportsDocValues()) {
       final Sort dvSort = new Sort(new SortField("title", SortField.Type.STRING));
-      dvSort.getSort()[0].setUseIndexValues(true);
       int hitCount2 = s.search(q, null, 10, dvSort).totalHits;
       assertEquals(hitCount, hitCount2);
     }

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocs.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocs.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocs.java Fri Feb  8 03:26:14 2013
@@ -36,7 +36,7 @@ import java.util.zip.GZIPInputStream;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
-import org.apache.lucene.document.SortedBytesDocValuesField;
+import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
 
@@ -184,7 +184,7 @@ public class LineFileDocs implements Clo
       doc.add(date);
 
       if (useDocValues) {
-        titleDV = new SortedBytesDocValuesField("titleDV", new BytesRef());
+        titleDV = new SortedDocValuesField("titleDV", new BytesRef());
         doc.add(titleDV);
       } else {
         titleDV = null;

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java Fri Feb  8 03:26:14 2013
@@ -254,6 +254,9 @@ public abstract class LuceneTestCase ext
 
   /** Gets the postingsFormat to run tests with. */
   public static final String TEST_POSTINGSFORMAT = System.getProperty("tests.postingsformat", "random");
+  
+  /** Gets the docValuesFormat to run tests with */
+  public static final String TEST_DOCVALUESFORMAT = System.getProperty("tests.docvaluesformat", "random");
 
   /** Gets the directory to run tests with */
   public static final String TEST_DIRECTORY = System.getProperty("tests.directory", "random");
@@ -1248,6 +1251,9 @@ public abstract class LuceneTestCase ext
       if (maybeWrap) {
         r = maybeWrapReader(r);
       }
+      if (r instanceof AtomicReader) {
+        _TestUtil.checkReader((AtomicReader)r);
+      }
       IndexSearcher ret = random.nextBoolean() ? new AssertingIndexSearcher(random, r) : new AssertingIndexSearcher(random, r.getContext());
       ret.setSimilarity(classEnvRule.similarity);
       return ret;

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/RunListenerPrintReproduceInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/RunListenerPrintReproduceInfo.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/RunListenerPrintReproduceInfo.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/RunListenerPrintReproduceInfo.java Fri Feb  8 03:26:14 2013
@@ -150,6 +150,7 @@ public final class RunListenerPrintRepro
     // Codec, postings, directories.
     if (!TEST_CODEC.equals("random")) addVmOpt(b, "tests.codec", TEST_CODEC);
     if (!TEST_POSTINGSFORMAT.equals("random")) addVmOpt(b, "tests.postingsformat", TEST_POSTINGSFORMAT);
+    if (!TEST_DOCVALUESFORMAT.equals("random")) addVmOpt(b, "tests.docvaluesformat", TEST_DOCVALUESFORMAT);
     if (!TEST_DIRECTORY.equals("random")) addVmOpt(b, "tests.directory", TEST_DIRECTORY);
 
     // Environment.

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java Fri Feb  8 03:26:14 2013
@@ -28,12 +28,15 @@ import java.util.Set;
 import java.util.TimeZone;
 
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.appending.AppendingRWCodec;
 import org.apache.lucene.codecs.asserting.AssertingCodec;
 import org.apache.lucene.codecs.lucene3x.PreFlexRWCodec;
+import org.apache.lucene.codecs.cheapbastard.CheapBastardCodec;
 import org.apache.lucene.codecs.compressing.CompressingCodec;
 import org.apache.lucene.codecs.lucene40.Lucene40Codec;
+import org.apache.lucene.codecs.lucene40.Lucene40RWCodec;
 import org.apache.lucene.codecs.lucene40.Lucene40RWPostingsFormat;
 import org.apache.lucene.codecs.lucene41.Lucene41Codec;
 import org.apache.lucene.codecs.lucene41.Lucene41RWCodec;
@@ -140,6 +143,7 @@ final class TestRuleSetupAndRestoreClass
     int randomVal = random.nextInt(10);
     if ("Lucene3x".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) &&
                                           "random".equals(TEST_POSTINGSFORMAT) &&
+                                          "random".equals(TEST_DOCVALUESFORMAT) &&
                                           randomVal == 0 &&
                                           !shouldAvoidCodec("Lucene3x"))) { // preflex-only setup
       codec = Codec.forName("Lucene3x");
@@ -147,23 +151,38 @@ final class TestRuleSetupAndRestoreClass
       PREFLEX_IMPERSONATION_IS_ACTIVE = true;
     } else if ("Lucene40".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) &&
                                                  "random".equals(TEST_POSTINGSFORMAT) &&
-                                                  randomVal < 3 &&
+                                                  randomVal == 2 &&
                                                   !shouldAvoidCodec("Lucene40"))) { // 4.0 setup
       codec = Codec.forName("Lucene40");
+      assert codec instanceof Lucene40RWCodec : "fix your classpath to have tests-framework.jar before lucene-core.jar";
       assert (PostingsFormat.forName("Lucene40") instanceof Lucene40RWPostingsFormat) : "fix your classpath to have tests-framework.jar before lucene-core.jar";
     } else if ("Lucene41".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) &&
                                                  "random".equals(TEST_POSTINGSFORMAT) &&
+                                                 "random".equals(TEST_DOCVALUESFORMAT) &&
                                                  randomVal == 1 &&
                                                  !shouldAvoidCodec("Lucene41"))) { 
       codec = Codec.forName("Lucene41");
       assert codec instanceof Lucene41RWCodec : "fix your classpath to have tests-framework.jar before lucene-core.jar";
-    } else if (!"random".equals(TEST_POSTINGSFORMAT)) {
+    } else if (("random".equals(TEST_POSTINGSFORMAT) == false) || ("random".equals(TEST_DOCVALUESFORMAT) == false)) {
+      // the user wired postings or DV: this is messy
+      // refactor into RandomCodec....
+      
       final PostingsFormat format;
-      if ("MockRandom".equals(TEST_POSTINGSFORMAT)) {
-        format = new MockRandomPostingsFormat(random);
+      if ("random".equals(TEST_POSTINGSFORMAT)) {
+        format = PostingsFormat.forName("Lucene41");
       } else {
         format = PostingsFormat.forName(TEST_POSTINGSFORMAT);
       }
+      
+      final DocValuesFormat dvFormat;
+      if ("random".equals(TEST_DOCVALUESFORMAT)) {
+        // pick one from SPI
+        String formats[] = DocValuesFormat.availableDocValuesFormats().toArray(new String[0]);
+        dvFormat = DocValuesFormat.forName(formats[random.nextInt(formats.length)]);
+      } else {
+        dvFormat = DocValuesFormat.forName(TEST_DOCVALUESFORMAT);
+      }
+      
       codec = new Lucene42Codec() {       
         @Override
         public PostingsFormat getPostingsFormatForField(String field) {
@@ -171,17 +190,24 @@ final class TestRuleSetupAndRestoreClass
         }
 
         @Override
+        public DocValuesFormat getDocValuesFormatForField(String field) {
+          return dvFormat;
+        }
+
+        @Override
         public String toString() {
-          return super.toString() + ": " + format.toString();
+          return super.toString() + ": " + format.toString() + ", " + dvFormat.toString();
         }
       };
     } else if ("SimpleText".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 9 && !shouldAvoidCodec("SimpleText"))) {
       codec = new SimpleTextCodec();
     } else if ("Appending".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 8 && !shouldAvoidCodec("Appending"))) {
       codec = new AppendingRWCodec();
-    } else if ("Asserting".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 7 && !shouldAvoidCodec("Asserting"))) {
+    } else if ("CheapBastard".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 7 && !shouldAvoidCodec("CheapBastard"))) {
+      codec = new CheapBastardCodec();
+    } else if ("Asserting".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 6 && !shouldAvoidCodec("Asserting"))) {
       codec = new AssertingCodec();
-    } else if ("Compressing".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 6 && !shouldAvoidCodec("Compressing"))) {
+    } else if ("Compressing".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Compressing"))) {
       codec = CompressingCodec.randomInstance(random);
     } else if (!"random".equals(TEST_CODEC)) {
       codec = Codec.forName(TEST_CODEC);

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java Fri Feb  8 03:26:14 2013
@@ -43,26 +43,26 @@ import java.util.zip.ZipEntry;
 import java.util.zip.ZipFile;
 
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.lucene42.Lucene42Codec;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
-import org.apache.lucene.document.ByteDocValuesField;
-import org.apache.lucene.document.DerefBytesDocValuesField;
+import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.DoubleDocValuesField;
 import org.apache.lucene.document.Field;
-import org.apache.lucene.document.FloatDocValuesField;
-import org.apache.lucene.document.IntDocValuesField;
-import org.apache.lucene.document.LongDocValuesField;
-import org.apache.lucene.document.PackedLongDocValuesField;
-import org.apache.lucene.document.ShortDocValuesField;
-import org.apache.lucene.document.SortedBytesDocValuesField;
-import org.apache.lucene.document.StraightBytesDocValuesField;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedDocValuesField;
+import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.CheckIndex;
+import org.apache.lucene.index.CheckIndex.Status.DocValuesStatus;
+import org.apache.lucene.index.CheckIndex.Status.FieldNormStatus;
+import org.apache.lucene.index.CheckIndex.Status.StoredFieldStatus;
+import org.apache.lucene.index.CheckIndex.Status.TermIndexStatus;
+import org.apache.lucene.index.CheckIndex.Status.TermVectorStatus;
 import org.apache.lucene.index.ConcurrentMergeScheduler;
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.IndexReader;
@@ -77,15 +77,14 @@ import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.TieredMergePolicy;
 import org.apache.lucene.search.FieldDoc;
-import org.apache.lucene.search.FilteredQuery;
 import org.apache.lucene.search.FilteredQuery.FilterStrategy;
+import org.apache.lucene.search.FilteredQuery;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.junit.Assert;
-
 import com.carrotsearch.randomizedtesting.RandomizedContext;
 import com.carrotsearch.randomizedtesting.generators.RandomInts;
 import com.carrotsearch.randomizedtesting.generators.RandomPicks;
@@ -226,6 +225,37 @@ public class _TestUtil {
       return indexStatus;
     }
   }
+  
+  /** This runs the CheckIndex tool on the Reader.  If any
+   *  issues are hit, a RuntimeException is thrown */
+  public static void checkReader(AtomicReader reader) throws IOException {
+    checkReader(reader, true);
+  }
+  
+  public static void checkReader(AtomicReader reader, boolean crossCheckTermVectors) throws IOException {
+    ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
+    PrintStream infoStream = new PrintStream(bos, false, "UTF-8");
+
+    FieldNormStatus fieldNormStatus = CheckIndex.testFieldNorms(reader, infoStream);
+    TermIndexStatus termIndexStatus = CheckIndex.testPostings(reader, infoStream);
+    StoredFieldStatus storedFieldStatus = CheckIndex.testStoredFields(reader, infoStream);
+    TermVectorStatus termVectorStatus = CheckIndex.testTermVectors(reader, infoStream, false, crossCheckTermVectors);
+    DocValuesStatus docValuesStatus = CheckIndex.testDocValues(reader, infoStream);
+    
+    if (fieldNormStatus.error != null || 
+      termIndexStatus.error != null ||
+      storedFieldStatus.error != null ||
+      termVectorStatus.error != null ||
+      docValuesStatus.error != null) {
+      System.out.println("CheckReader failed");
+      System.out.println(bos.toString("UTF-8"));
+      throw new RuntimeException("CheckReader failed");
+    } else {
+      if (LuceneTestCase.INFOSTREAM) {
+        System.out.println(bos.toString("UTF-8"));
+      }
+    }
+  }
 
   // NOTE: only works for TMP and LMP!!
   public static void setUseCompoundFile(MergePolicy mp, boolean v) {
@@ -233,6 +263,8 @@ public class _TestUtil {
       ((TieredMergePolicy) mp).setUseCompoundFile(v);
     } else if (mp instanceof LogMergePolicy) {
       ((LogMergePolicy) mp).setUseCompoundFile(v);
+    } else {
+      throw new IllegalArgumentException("cannot set compound file for MergePolicy " + mp);
     }
   }
 
@@ -676,6 +708,24 @@ public class _TestUtil {
       }
     };
   }
+  
+  /** Return a Codec that can read any of the
+   *  default codecs and formats, but always writes in the specified
+   *  format. */
+  public static Codec alwaysDocValuesFormat(final DocValuesFormat format) {
+    // TODO: we really need for docvalues impls etc to announce themselves
+    // (and maybe their params, too) to infostream on flush and merge.
+    // otherwise in a real debugging situation we won't know whats going on!
+    if (LuceneTestCase.VERBOSE) {
+      System.out.println("forcing docvalues format to:" + format);
+    }
+    return new Lucene42Codec() {
+      @Override
+      public DocValuesFormat getDocValuesFormatForField(String field) {
+        return format;
+      }
+    };
+  }
 
   // TODO: generalize all 'test-checks-for-crazy-codecs' to
   // annotations (LUCENE-3489)
@@ -825,50 +875,20 @@ public class _TestUtil {
     for(IndexableField f : doc1) {
       final Field field1 = (Field) f;
       final Field field2;
-      final DocValues.Type dvType = field1.fieldType().docValueType();
+      final DocValuesType dvType = field1.fieldType().docValueType();
       if (dvType != null) {
         switch(dvType) {
-        case VAR_INTS:
-          field2 = new PackedLongDocValuesField(field1.name(), field1.numericValue().longValue());
-          break;
-        case FIXED_INTS_8:
-          field2 = new ByteDocValuesField(field1.name(), field1.numericValue().byteValue());
-          break;
-        case FIXED_INTS_16:
-          field2 = new ShortDocValuesField(field1.name(), field1.numericValue().shortValue());
-          break;
-        case FIXED_INTS_32:
-          field2 = new IntDocValuesField(field1.name(), field1.numericValue().intValue());
-          break;
-        case FIXED_INTS_64:
-          field2 = new LongDocValuesField(field1.name(), field1.numericValue().longValue());
-          break;
-        case FLOAT_32:
-          field2 = new FloatDocValuesField(field1.name(), field1.numericValue().floatValue());
-          break;
-        case FLOAT_64:
-          field2 = new DoubleDocValuesField(field1.name(), field1.numericValue().doubleValue());
-          break;
-        case BYTES_FIXED_STRAIGHT:
-          field2 = new StraightBytesDocValuesField(field1.name(), field1.binaryValue(), true);
-          break;
-        case BYTES_VAR_STRAIGHT:
-          field2 = new StraightBytesDocValuesField(field1.name(), field1.binaryValue(), false);
-          break;
-        case BYTES_FIXED_DEREF:
-          field2 = new DerefBytesDocValuesField(field1.name(), field1.binaryValue(), true);
-          break;
-        case BYTES_VAR_DEREF:
-          field2 = new DerefBytesDocValuesField(field1.name(), field1.binaryValue(), false);
-          break;
-        case BYTES_FIXED_SORTED:
-          field2 = new SortedBytesDocValuesField(field1.name(), field1.binaryValue(), true);
-          break;
-        case BYTES_VAR_SORTED:
-          field2 = new SortedBytesDocValuesField(field1.name(), field1.binaryValue(), false);
+          case NUMERIC:
+            field2 = new NumericDocValuesField(field1.name(), field1.numericValue().longValue());
+            break;
+          case BINARY:
+            field2 = new BinaryDocValuesField(field1.name(), field1.binaryValue());
           break;
-        default:
-          throw new IllegalStateException("unknown Type: " + dvType);
+          case SORTED:
+            field2 = new SortedDocValuesField(field1.name(), field1.binaryValue());
+            break;
+          default:
+            throw new IllegalStateException("unknown Type: " + dvType);
         }
       } else {
         field2 = new Field(field1.name(), field1.stringValue(), field1.fieldType());

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec Fri Feb  8 03:26:14 2013
@@ -16,8 +16,10 @@
 org.apache.lucene.codecs.lucene3x.PreFlexRWCodec
 org.apache.lucene.codecs.appending.AppendingRWCodec
 org.apache.lucene.codecs.asserting.AssertingCodec
+org.apache.lucene.codecs.cheapbastard.CheapBastardCodec
 org.apache.lucene.codecs.compressing.FastCompressingCodec
 org.apache.lucene.codecs.compressing.FastDecompressionCompressingCodec
 org.apache.lucene.codecs.compressing.HighCompressionCompressingCodec
 org.apache.lucene.codecs.compressing.dummy.DummyCompressingCodec
+org.apache.lucene.codecs.lucene40.Lucene40RWCodec
 org.apache.lucene.codecs.lucene41.Lucene41RWCodec

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java Fri Feb  8 03:26:14 2013
@@ -55,6 +55,7 @@ public class SchemaCodecFactory extends 
         }
         return super.getPostingsFormatForField(field);
       }
+      // TODO: when dv support is added to solr, add it here too
     };
   }
 

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java Fri Feb  8 03:26:14 2013
@@ -36,6 +36,7 @@ import org.apache.lucene.analysis.util.T
 import org.apache.lucene.analysis.util.TokenizerFactory;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.analysis.util.WordlistLoader;
 import org.apache.solr.common.ResourceLoader;
 import org.apache.solr.handler.admin.CoreAdminHandler;
@@ -178,6 +179,7 @@ public class SolrResourceLoader implemen
   void reloadLuceneSPI() {
     // Codecs:
     PostingsFormat.reloadPostingsFormats(this.classLoader);
+    DocValuesFormat.reloadDocValuesFormats(this.classLoader);
     Codec.reloadCodecs(this.classLoader);
     // Analysis:
     CharFilterFactory.reloadCharFilters(this.classLoader);

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/component/FieldFacetStats.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/component/FieldFacetStats.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/component/FieldFacetStats.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/component/FieldFacetStats.java Fri Feb  8 03:26:14 2013
@@ -16,16 +16,17 @@ package org.apache.solr.handler.componen
  * limitations under the License.
  */
 
-import org.apache.lucene.search.FieldCache;
-import org.apache.lucene.util.BytesRef;
-import org.apache.solr.schema.FieldType;
-import org.apache.solr.schema.SchemaField;
-
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.search.FieldCache;
+import org.apache.lucene.util.BytesRef;
+import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.SchemaField;
+
 
 /**
  * FieldFacetStats is a utility to accumulate statistics on a set of values in one field,
@@ -39,7 +40,7 @@ import java.util.Map;
 
 public class FieldFacetStats {
   public final String name;
-  final FieldCache.DocTermsIndex si;
+  final SortedDocValues si;
   final SchemaField facet_sf;
   final SchemaField field_sf;
 
@@ -55,15 +56,15 @@ public class FieldFacetStats {
 
   private final BytesRef tempBR = new BytesRef();
 
-  public FieldFacetStats(String name, FieldCache.DocTermsIndex si, SchemaField field_sf, SchemaField facet_sf, int numStatsTerms) {
+  public FieldFacetStats(String name, SortedDocValues si, SchemaField field_sf, SchemaField facet_sf, int numStatsTerms) {
     this.name = name;
     this.si = si;
     this.field_sf = field_sf;
     this.facet_sf = facet_sf;
     this.numStatsTerms = numStatsTerms;
 
-    startTermIndex = 1;
-    endTermIndex = si.numOrd();
+    startTermIndex = 0;
+    endTermIndex = si.getValueCount();
     nTerms = endTermIndex - startTermIndex;
 
     facetStatsValues = new HashMap<String, StatsValues>();
@@ -79,10 +80,11 @@ public class FieldFacetStats {
 
   BytesRef getTermText(int docID, BytesRef ret) {
     final int ord = si.getOrd(docID);
-    if (ord == 0) {
+    if (ord == -1) {
       return null;
     } else {
-      return si.lookup(ord, ret);
+      si.lookupOrd(ord, ret);
+      return ret;
     }
   }
 
@@ -90,7 +92,14 @@ public class FieldFacetStats {
     int term = si.getOrd(docID);
     int arrIdx = term - startTermIndex;
     if (arrIdx >= 0 && arrIdx < nTerms) {
-      final BytesRef br = si.lookup(term, tempBR);
+      
+      final BytesRef br;
+      if (term == -1) {
+        br = null;
+      } else {
+        br = tempBR;
+        si.lookupOrd(term, tempBR);
+      }
       String key = (br == null)?null:facet_sf.getType().indexedToReadable(br.utf8ToString());
       StatsValues stats = facetStatsValues.get(key);
       if (stats == null) {
@@ -117,7 +126,13 @@ public class FieldFacetStats {
     int term = si.getOrd(docID);
     int arrIdx = term - startTermIndex;
     if (arrIdx >= 0 && arrIdx < nTerms) {
-      final BytesRef br = si.lookup(term, tempBR);
+      final BytesRef br;
+      if (term == -1) {
+        br = null;
+      } else {
+        br = tempBR;
+        si.lookupOrd(term, tempBR);
+      }
       String key = br == null ? null : br.utf8ToString();
       HashMap<String, Integer> statsTermCounts = facetStatsTerms.get(statsTermNum);
       Integer statsTermCount = statsTermCounts.get(key);