You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by yo...@apache.org on 2011/11/05 13:47:05 UTC

svn commit: r1197949 [4/7] - in /lucene/dev/branches/solrcloud: ./ dev-tools/eclipse/ dev-tools/idea/.idea/ dev-tools/idea/lucene/ dev-tools/idea/lucene/contrib/ dev-tools/idea/lucene/contrib/misc/ dev-tools/maven/ dev-tools/maven/lucene/contrib/misc/ ...

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java Sat Nov  5 12:46:53 2011
@@ -68,14 +68,14 @@ public class FixedGapTermsIndexReader ex
   // start of the field info data
   protected long dirOffset;
 
-  public FixedGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, Comparator<BytesRef> termComp, int codecId, IOContext context)
+  public FixedGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, Comparator<BytesRef> termComp, String segmentSuffix, IOContext context)
     throws IOException {
 
     this.termComp = termComp;
 
     assert indexDivisor == -1 || indexDivisor > 0;
 
-    in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION), context);
+    in = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION), context);
     
     boolean success = false;
 
@@ -387,17 +387,8 @@ public class FixedGapTermsIndexReader ex
     }
   }
 
-  public static void files(Directory dir, SegmentInfo info, int id, Collection<String> files) {
-    files.add(IndexFileNames.segmentFileName(info.name, id, FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
-  }
-
-  public static void getIndexExtensions(Collection<String> extensions) {
-    extensions.add(FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION);
-  }
-
-  @Override
-  public void getExtensions(Collection<String> extensions) {
-    getIndexExtensions(extensions);
+  public static void files(Directory dir, SegmentInfo info, String segmentSuffix, Collection<String> files) {
+    files.add(IndexFileNames.segmentFileName(info.name, segmentSuffix, FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
   }
 
   @Override

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexWriter.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexWriter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexWriter.java Sat Nov  5 12:46:53 2011
@@ -56,7 +56,7 @@ public class FixedGapTermsIndexWriter ex
   private final FieldInfos fieldInfos; // unread
 
   public FixedGapTermsIndexWriter(SegmentWriteState state) throws IOException {
-    final String indexFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_INDEX_EXTENSION);
+    final String indexFileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, TERMS_INDEX_EXTENSION);
     termIndexInterval = state.termIndexInterval;
     out = state.directory.createOutput(indexFileName, state.context);
     boolean success = false;

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/MultiLevelSkipListReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/MultiLevelSkipListReader.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/MultiLevelSkipListReader.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/MultiLevelSkipListReader.java Sat Nov  5 12:46:53 2011
@@ -255,6 +255,7 @@ public abstract class MultiLevelSkipList
     private int pos;
     
     SkipBuffer(IndexInput input, int length) throws IOException {
+      super("SkipBuffer on " + input);
       data = new byte[length];
       pointer = input.getFilePointer();
       input.readBytes(data, 0, length);

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/PerDocConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/PerDocConsumer.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/PerDocConsumer.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/PerDocConsumer.java Sat Nov  5 12:46:53 2011
@@ -29,7 +29,7 @@ import org.apache.lucene.index.values.Va
  * this convert field values into a Codec specific format during indexing.
  * <p>
  * The {@link PerDocConsumer} API is accessible through the
- * {@link Codec} - API providing per field consumers and producers for inverted
+ * {@link PostingsFormat} - API providing per field consumers and producers for inverted
  * data (terms, postings) as well as per-document data.
  * 
  * @lucene.experimental

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/PerDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/PerDocValues.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/PerDocValues.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/PerDocValues.java Sat Nov  5 12:46:53 2011
@@ -28,7 +28,7 @@ import org.apache.lucene.index.values.In
  * {@link PerDocConsumer} counterpart.
  * <p>
  * The {@link PerDocValues} API is accessible through the
- * {@link Codec} - API providing per field consumers and producers for inverted
+ * {@link PostingsFormat} - API providing per field consumers and producers for inverted
  * data (terms, postings) as well as per-document data.
  * 
  * @lucene.experimental

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/PostingsReaderBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/PostingsReaderBase.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/PostingsReaderBase.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/PostingsReaderBase.java Sat Nov  5 12:46:53 2011
@@ -36,6 +36,9 @@ import org.apache.lucene.util.Bits;
  *  time. 
  *  @lucene.experimental */
 
+// TODO: find a better name; this defines the API that the
+// terms dict impls use to talk to a postings impl.
+// TermsDict + PostingsReader/WriterBase == PostingsConsumer/Producer
 public abstract class PostingsReaderBase implements Closeable {
 
   public abstract void init(IndexInput termsIn) throws IOException;

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/PostingsWriterBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/PostingsWriterBase.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/PostingsWriterBase.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/PostingsWriterBase.java Sat Nov  5 12:46:53 2011
@@ -27,6 +27,9 @@ import org.apache.lucene.index.FieldInfo
  * @lucene.experimental
  */
 
+// TODO: find a better name; this defines the API that the
+// terms dict impls use to talk to a postings impl.
+// TermsDict + PostingsReader/WriterBase == PostingsConsumer/Producer
 public abstract class PostingsWriterBase extends PostingsConsumer implements Closeable {
 
   public abstract void start(IndexOutput termsOut) throws IOException;

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/SegmentInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/SegmentInfosReader.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/SegmentInfosReader.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/SegmentInfosReader.java Sat Nov  5 12:46:53 2011
@@ -20,6 +20,7 @@ package org.apache.lucene.index.codecs;
 import java.io.IOException;
 
 import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 
@@ -33,9 +34,9 @@ public abstract class SegmentInfosReader
    * Read {@link SegmentInfos} data from a directory.
    * @param directory directory to read from
    * @param segmentsFileName name of the "segments_N" file
-   * @param codecs current codecs
+   * @param header input of "segments_N" file after reading preamble
    * @param infos empty instance to be populated with data
    * @throws IOException
    */
-  public abstract void read(Directory directory, String segmentsFileName, CodecProvider codecs, SegmentInfos infos, IOContext context) throws IOException;
+  public abstract void read(Directory directory, String segmentsFileName, ChecksumIndexInput header, SegmentInfos infos, IOContext context) throws IOException;
 }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/SegmentInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/SegmentInfosWriter.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/SegmentInfosWriter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/SegmentInfosWriter.java Sat Nov  5 12:46:53 2011
@@ -42,13 +42,13 @@ public abstract class SegmentInfosWriter
    * phase commit" operations as described above.
    * @throws IOException
    */
-  public abstract IndexOutput writeInfos(Directory dir, String segmentsFileName, SegmentInfos infos, IOContext context) throws IOException;
+  public abstract IndexOutput writeInfos(Directory dir, String segmentsFileName, String codecID, SegmentInfos infos, IOContext context) throws IOException;
   
   /**
    * First phase of the two-phase commit - ensure that all output can be
    * successfully written out.
    * @param out an instance of {@link IndexOutput} returned from a previous
-   * call to {@link #writeInfos(Directory, String, SegmentInfos, IOContext)}.
+   * call to {@link #writeInfos(Directory, String, String, SegmentInfos, IOContext)}.
    * @throws IOException
    */
   public abstract void prepareCommit(IndexOutput out) throws IOException;
@@ -57,7 +57,7 @@ public abstract class SegmentInfosWriter
    * Second phase of the two-phase commit. In this step the output should be
    * finalized and closed.
    * @param out an instance of {@link IndexOutput} returned from a previous
-   * call to {@link #writeInfos(Directory, String, SegmentInfos, IOContext)}.
+   * call to {@link #writeInfos(Directory, String, String, SegmentInfos, IOContext)}.
    * @throws IOException
    */
   public abstract void finishCommit(IndexOutput out) throws IOException;

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/TermsIndexReaderBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/TermsIndexReaderBase.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/TermsIndexReaderBase.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/TermsIndexReaderBase.java Sat Nov  5 12:46:53 2011
@@ -45,8 +45,6 @@ public abstract class TermsIndexReaderBa
 
   public abstract void close() throws IOException;
 
-  public abstract void getExtensions(Collection<String> extensions);
-
   public abstract boolean supportsOrd();
 
   public abstract int getDivisor();

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java Sat Nov  5 12:46:53 2011
@@ -57,9 +57,9 @@ public class VariableGapTermsIndexReader
   protected long dirOffset;
 
   final String segment;
-  public VariableGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, int codecId, IOContext context)
+  public VariableGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, String segmentSuffix, IOContext context)
     throws IOException {
-    in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION), new IOContext(context, true));
+    in = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION), new IOContext(context, true));
     this.segment = segment;
     boolean success = false;
     assert indexDivisor == -1 || indexDivisor > 0;
@@ -215,17 +215,8 @@ public class VariableGapTermsIndexReader
     }
   }
 
-  public static void files(Directory dir, SegmentInfo info, int id, Collection<String> files) {
-    files.add(IndexFileNames.segmentFileName(info.name, id, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
-  }
-
-  public static void getIndexExtensions(Collection<String> extensions) {
-    extensions.add(VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION);
-  }
-
-  @Override
-  public void getExtensions(Collection<String> extensions) {
-    getIndexExtensions(extensions);
+  public static void files(Directory dir, SegmentInfo info, String segmentSuffix, Collection<String> files) {
+    files.add(IndexFileNames.segmentFileName(info.name, segmentSuffix, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
   }
 
   @Override

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexWriter.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexWriter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexWriter.java Sat Nov  5 12:46:53 2011
@@ -158,7 +158,7 @@ public class VariableGapTermsIndexWriter
   // in the extremes.
 
   public VariableGapTermsIndexWriter(SegmentWriteState state, IndexTermSelector policy) throws IOException {
-    final String indexFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_INDEX_EXTENSION);
+    final String indexFileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, TERMS_INDEX_EXTENSION);
     out = state.directory.createOutput(indexFileName, state.context);
     boolean success = false;
     try {

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/sep/SepDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/sep/SepDocValuesConsumer.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/sep/SepDocValuesConsumer.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/sep/SepDocValuesConsumer.java Sat Nov  5 12:46:53 2011
@@ -47,12 +47,12 @@ public class SepDocValuesConsumer extend
   }
 
   @SuppressWarnings("fallthrough")
-  public static void files(Directory dir, SegmentInfo segmentInfo, int codecId,
+  public static void files(Directory dir, SegmentInfo segmentInfo,
       Set<String> files) throws IOException {
     FieldInfos fieldInfos = segmentInfo.getFieldInfos();
     for (FieldInfo fieldInfo : fieldInfos) {
-      if (fieldInfo.getCodecId() == codecId && fieldInfo.hasDocValues()) {
-        String filename = docValuesId(segmentInfo.name, codecId, fieldInfo.number);
+      if (fieldInfo.hasDocValues()) {
+        String filename = docValuesId(segmentInfo.name, fieldInfo.number);
         switch (fieldInfo.getDocValues()) {
           case BYTES_FIXED_DEREF:
           case BYTES_VAR_DEREF:
@@ -83,9 +83,4 @@ public class SepDocValuesConsumer extend
       }
     }
   }
-
-  public static void getExtensions(Set<String> extensions) {
-    extensions.add(Writer.DATA_EXTENSION);
-    extensions.add(Writer.INDEX_EXTENSION);
-  }
 }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/sep/SepDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/sep/SepDocValuesProducer.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/sep/SepDocValuesProducer.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/sep/SepDocValuesProducer.java Sat Nov  5 12:46:53 2011
@@ -39,7 +39,7 @@ public class SepDocValuesProducer extend
    * {@link IndexDocValues} instances for this segment and codec.
    */
   public SepDocValuesProducer(SegmentReadState state) throws IOException {
-    docValues = load(state.fieldInfos, state.segmentInfo.name, state.segmentInfo.docCount, state.dir, state.codecId, state.context);
+    docValues = load(state.fieldInfos, state.segmentInfo.name, state.segmentInfo.docCount, state.dir, state.context);
   }
   
   @Override

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReader.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReader.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReader.java Sat Nov  5 12:46:53 2011
@@ -60,23 +60,23 @@ public class SepPostingsReader extends P
   int maxSkipLevels;
   int skipMinimum;
 
-  public SepPostingsReader(Directory dir, SegmentInfo segmentInfo, IOContext context, IntStreamFactory intFactory, int codecId) throws IOException {
+  public SepPostingsReader(Directory dir, SegmentInfo segmentInfo, IOContext context, IntStreamFactory intFactory, String segmentSuffix) throws IOException {
     boolean success = false;
     try {
 
-      final String docFileName = IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriter.DOC_EXTENSION);
+      final String docFileName = IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.DOC_EXTENSION);
       docIn = intFactory.openInput(dir, docFileName, context);
 
-      skipIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriter.SKIP_EXTENSION), context);
+      skipIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.SKIP_EXTENSION), context);
 
       if (segmentInfo.getFieldInfos().hasFreq()) {
-        freqIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriter.FREQ_EXTENSION), context);        
+        freqIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.FREQ_EXTENSION), context);        
       } else {
         freqIn = null;
       }
       if (segmentInfo.getHasProx()) {
-        posIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriter.POS_EXTENSION), context);
-        payloadIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriter.PAYLOAD_EXTENSION), context);
+        posIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.POS_EXTENSION), context);
+        payloadIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.PAYLOAD_EXTENSION), context);
       } else {
         posIn = null;
         payloadIn = null;
@@ -89,17 +89,17 @@ public class SepPostingsReader extends P
     }
   }
 
-  public static void files(SegmentInfo segmentInfo, int codecId, Collection<String> files) throws IOException {
-    files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriter.DOC_EXTENSION));
-    files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriter.SKIP_EXTENSION));
+  public static void files(SegmentInfo segmentInfo, String segmentSuffix, Collection<String> files) throws IOException {
+    files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.DOC_EXTENSION));
+    files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.SKIP_EXTENSION));
 
     if (segmentInfo.getFieldInfos().hasFreq()) {
-      files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriter.FREQ_EXTENSION));
+      files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.FREQ_EXTENSION));
     }
 
     if (segmentInfo.getHasProx()) {
-      files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriter.POS_EXTENSION));
-      files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriter.PAYLOAD_EXTENSION));
+      files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.POS_EXTENSION));
+      files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.PAYLOAD_EXTENSION));
     }
   }
 

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriter.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriter.java Sat Nov  5 12:46:53 2011
@@ -20,7 +20,6 @@ package org.apache.lucene.index.codecs.s
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Set;
 
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.DocsEnum;
@@ -116,27 +115,27 @@ public final class SepPostingsWriter ext
     try {
       this.skipInterval = skipInterval;
       this.skipMinimum = skipInterval; /* set to the same for now */
-      final String docFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, DOC_EXTENSION);
+      final String docFileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, DOC_EXTENSION);
       docOut = factory.createOutput(state.directory, docFileName, state.context);
       docIndex = docOut.index();
       
       if (state.fieldInfos.hasFreq()) {
-        final String frqFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, FREQ_EXTENSION);
+        final String frqFileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, FREQ_EXTENSION);
         freqOut = factory.createOutput(state.directory, frqFileName, state.context);
         freqIndex = freqOut.index();
       }
 
       if (state.fieldInfos.hasProx()) {      
-        final String posFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, POS_EXTENSION);
+        final String posFileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, POS_EXTENSION);
         posOut = factory.createOutput(state.directory, posFileName, state.context);
         posIndex = posOut.index();
         
         // TODO: -- only if at least one field stores payloads?
-        final String payloadFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, PAYLOAD_EXTENSION);
+        final String payloadFileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, PAYLOAD_EXTENSION);
         payloadOut = state.directory.createOutput(payloadFileName, state.context);
       }
       
-      final String skipFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, SKIP_EXTENSION);
+      final String skipFileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, SKIP_EXTENSION);
       skipOut = state.directory.createOutput(skipFileName, state.context);
       
       totalNumDocs = state.numDocs;
@@ -203,7 +202,7 @@ public final class SepPostingsWriter ext
     //System.out.println("SEPW: startDoc: write doc=" + docID + " delta=" + delta + " out.fp=" + docOut);
 
     if (docID < 0 || (df > 0 && delta <= 0)) {
-      throw new CorruptIndexException("docs out of order (" + docID + " <= " + lastDocID + " )");
+      throw new CorruptIndexException("docs out of order (" + docID + " <= " + lastDocID + " ) (docOut: " + docOut + ")");
     }
 
     if ((++df % skipInterval) == 0) {
@@ -391,12 +390,4 @@ public final class SepPostingsWriter ext
   public void close() throws IOException {
     IOUtils.close(docOut, skipOut, freqOut, posOut, payloadOut);
   }
-
-  public static void getExtensions(Set<String> extensions) {
-    extensions.add(DOC_EXTENSION);
-    extensions.add(FREQ_EXTENSION);
-    extensions.add(SKIP_EXTENSION);
-    extensions.add(POS_EXTENSION);
-    extensions.add(PAYLOAD_EXTENSION);
-  }
 }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java Sat Nov  5 12:46:53 2011
@@ -60,7 +60,7 @@ class SimpleTextFieldsReader extends Fie
   final static BytesRef PAYLOAD = SimpleTextFieldsWriter.PAYLOAD;
 
   public SimpleTextFieldsReader(SegmentReadState state) throws IOException {
-    in = state.dir.openInput(SimpleTextCodec.getPostingsFileName(state.segmentInfo.name, state.codecId), state.context);
+    in = state.dir.openInput(SimpleTextPostingsFormat.getPostingsFileName(state.segmentInfo.name, state.segmentSuffix), state.context);
    
     fieldInfos = state.fieldInfos;
   }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java Sat Nov  5 12:46:53 2011
@@ -47,7 +47,7 @@ class SimpleTextFieldsWriter extends Fie
   final static BytesRef PAYLOAD = new BytesRef("        payload ");
 
   public SimpleTextFieldsWriter(SegmentWriteState state) throws IOException {
-    final String fileName = SimpleTextCodec.getPostingsFileName(state.segmentName, state.codecId);
+    final String fileName = SimpleTextPostingsFormat.getPostingsFileName(state.segmentName, state.segmentSuffix);
     out = state.directory.createOutput(fileName, state.context);
   }
 

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/Bytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/Bytes.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/Bytes.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/Bytes.java Sat Nov  5 12:46:53 2011
@@ -61,6 +61,9 @@ import org.apache.lucene.util.packed.Pac
  * @lucene.experimental
  */
 public final class Bytes {
+
+  static final String DV_SEGMENT_SUFFIX = "dv";
+
   // TODO - add bulk copy where possible
   private Bytes() { /* don't instantiate! */
   }
@@ -244,7 +247,7 @@ public final class Bytes {
       if (datOut == null) {
         boolean success = false;
         try {
-          datOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
+          datOut = dir.createOutput(IndexFileNames.segmentFileName(id, DV_SEGMENT_SUFFIX,
               DATA_EXTENSION), context);
           CodecUtil.writeHeader(datOut, codecName, version);
           success = true;
@@ -269,7 +272,7 @@ public final class Bytes {
       boolean success = false;
       try {
         if (idxOut == null) {
-          idxOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
+          idxOut = dir.createOutput(IndexFileNames.segmentFileName(id, DV_SEGMENT_SUFFIX,
               INDEX_EXTENSION), context);
           CodecUtil.writeHeader(idxOut, codecName, version);
         }
@@ -307,10 +310,10 @@ public final class Bytes {
     @Override
     public void files(Collection<String> files) throws IOException {
       assert datOut != null;
-      files.add(IndexFileNames.segmentFileName(id, "", DATA_EXTENSION));
+      files.add(IndexFileNames.segmentFileName(id, DV_SEGMENT_SUFFIX, DATA_EXTENSION));
       if (idxOut != null) { // called after flush - so this must be initialized
         // if needed or present
-        final String idxFile = IndexFileNames.segmentFileName(id, "",
+        final String idxFile = IndexFileNames.segmentFileName(id, DV_SEGMENT_SUFFIX,
             INDEX_EXTENSION);
         files.add(idxFile);
       }
@@ -334,11 +337,11 @@ public final class Bytes {
       IndexInput indexIn = null;
       boolean success = false;
       try {
-        dataIn = dir.openInput(IndexFileNames.segmentFileName(id, "",
+        dataIn = dir.openInput(IndexFileNames.segmentFileName(id, DV_SEGMENT_SUFFIX,
                                                               Writer.DATA_EXTENSION), context);
         version = CodecUtil.checkHeader(dataIn, codecName, maxVersion, maxVersion);
         if (doIndex) {
-          indexIn = dir.openInput(IndexFileNames.segmentFileName(id, "",
+          indexIn = dir.openInput(IndexFileNames.segmentFileName(id, DV_SEGMENT_SUFFIX,
                                                                  Writer.INDEX_EXTENSION), context);
           final int version2 = CodecUtil.checkHeader(indexIn, codecName,
                                                      maxVersion, maxVersion);
@@ -494,8 +497,7 @@ public final class Bytes {
     
     protected void releaseResources() {
       hash.close();
-      bytesUsed
-      .addAndGet((-docToEntry.length) * RamUsageEstimator.NUM_BYTES_INT);
+      bytesUsed.addAndGet((-docToEntry.length) * RamUsageEstimator.NUM_BYTES_INT);
       docToEntry = null;
     }
     

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/IndexDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/IndexDocValues.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/IndexDocValues.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/IndexDocValues.java Sat Nov  5 12:46:53 2011
@@ -24,8 +24,7 @@ import org.apache.lucene.document.IndexD
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.codecs.Codec;
-import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.DocValuesFormat;
 import org.apache.lucene.util.BytesRef;
 
 /**
@@ -40,13 +39,11 @@ import org.apache.lucene.util.BytesRef;
  * performance {@link IndexDocValues} should be consumed per-segment just like
  * IndexReader.
  * <p>
- * {@link IndexDocValues} are fully integrated into the {@link Codec} API.
- * Custom implementations can be exposed on a per field basis via
- * {@link CodecProvider}.
+ * {@link IndexDocValues} are fully integrated into the {@link DocValuesFormat} API.
  * 
  * @see ValueType for limitations and default implementation documentation
  * @see IndexDocValuesField for adding values to the index
- * @see Codec#docsConsumer(org.apache.lucene.index.PerDocWriteState) for
+ * @see DocValuesFormat#docsConsumer(org.apache.lucene.index.PerDocWriteState) for
  *      customization
  * @lucene.experimental
  */

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/PackedIntValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/PackedIntValues.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/PackedIntValues.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/PackedIntValues.java Sat Nov  5 12:46:53 2011
@@ -156,7 +156,7 @@ class PackedIntValues {
     protected PackedIntsReader(Directory dir, String id, int numDocs,
         IOContext context) throws IOException {
       datIn = dir.openInput(
-          IndexFileNames.segmentFileName(id, "", Writer.DATA_EXTENSION),
+                IndexFileNames.segmentFileName(id, Bytes.DV_SEGMENT_SUFFIX, Writer.DATA_EXTENSION),
           context);
       this.numDocs = numDocs;
       boolean success = false;

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/ValueType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/ValueType.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/ValueType.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/ValueType.java Sat Nov  5 12:46:53 2011
@@ -17,7 +17,7 @@ package org.apache.lucene.index.values;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.DocValuesFormat;
 import org.apache.lucene.index.values.IndexDocValues.SortedSource;
 import org.apache.lucene.index.values.IndexDocValues.Source;
 import org.apache.lucene.util.BytesRef;
@@ -27,7 +27,7 @@ import org.apache.lucene.util.packed.Pac
  * <code>ValueType</code> specifies the {@link IndexDocValues} type for a
  * certain field. A <code>ValueType</code> only defines the data type for a field
  * while the actual implementation used to encode and decode the values depends
- * on the the {@link Codec#docsConsumer} and {@link Codec#docsProducer} methods.
+ * on the the {@link DocValuesFormat#docsConsumer} and {@link DocValuesFormat#docsProducer} methods.
  * 
  * @lucene.experimental
  */

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/IndexSearcher.java Sat Nov  5 12:46:53 2011
@@ -35,14 +35,18 @@ import org.apache.lucene.index.CorruptIn
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader.ReaderContext;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.index.StoredFieldVisitor;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
 import org.apache.lucene.search.similarities.DefaultSimilarityProvider;
 import org.apache.lucene.search.similarities.SimilarityProvider;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.NIOFSDirectory;    // javadoc
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.ReaderUtil;
+import org.apache.lucene.util.TermContext;
 import org.apache.lucene.util.ThreadInterruptedException;
 
 /** Implements search over a single IndexReader.
@@ -211,36 +215,6 @@ public class IndexSearcher implements Cl
     return reader;
   }
 
-  /** Expert: Returns one greater than the largest possible document number.
-   * 
-   * @see org.apache.lucene.index.IndexReader#maxDoc()
-   */
-  public int maxDoc() {
-    return reader.maxDoc();
-  }
-
-  /** Returns total docFreq for this term. */
-  public int docFreq(final Term term) throws IOException {
-    if (executor == null) {
-      return reader.docFreq(term);
-    } else {
-      final ExecutionHelper<Integer> runner = new ExecutionHelper<Integer>(executor);
-      for(int i = 0; i < leafContexts.length; i++) {
-        final IndexReader leaf = leafContexts[i].reader;
-        runner.submit(new Callable<Integer>() {
-            public Integer call() throws IOException {
-              return Integer.valueOf(leaf.docFreq(term));
-            }
-          });
-      }
-      int docFreq = 0;
-      for (Integer num : runner) {
-        docFreq += num.intValue();
-      }
-      return docFreq;
-    }
-  }
-
   /* Sugar for <code>.getIndexReader().document(docID)</code> */
   public Document doc(int docID) throws CorruptIndexException, IOException {
     return reader.document(docID);
@@ -860,4 +834,34 @@ public class IndexSearcher implements Cl
   public String toString() {
     return "IndexSearcher(" + reader + "; executor=" + executor + ")";
   }
+  
+  /**
+   * Returns {@link TermStatistics} for a term
+   * @lucene.experimental
+   */
+  public TermStatistics termStatistics(Term term, TermContext context) throws IOException {
+    return new TermStatistics(term.bytes(), context.docFreq(), context.totalTermFreq());
+  };
+  
+  /**
+   * Returns {@link CollectionStatistics} for a field
+   * @lucene.experimental
+   */
+  public CollectionStatistics collectionStatistics(String field) throws IOException {
+    final int docCount;
+    final long sumTotalTermFreq;
+    final long sumDocFreq;
+    
+    Terms terms = MultiFields.getTerms(reader, field);
+    if (terms == null) {
+      docCount = 0;
+      sumTotalTermFreq = 0;
+      sumDocFreq = 0;
+    } else {
+      docCount = terms.getDocCount();
+      sumTotalTermFreq = terms.getSumTotalTermFreq();
+      sumDocFreq = terms.getSumDocFreq();
+    }
+    return new CollectionStatistics(field, reader.maxDoc(), docCount, sumTotalTermFreq, sumDocFreq);
+  }
 }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java Sat Nov  5 12:46:53 2011
@@ -141,13 +141,15 @@ public class MultiPhraseQuery extends Qu
       final ReaderContext context = searcher.getTopReaderContext();
       
       // compute idf
-      ArrayList<TermContext> allTerms = new ArrayList<TermContext>();
+      ArrayList<TermStatistics> allTermStats = new ArrayList<TermStatistics>();
       for(final Term[] terms: termArrays) {
         for (Term term: terms) {
-          allTerms.add(TermContext.build(context, term, true));
+          TermContext termContext = TermContext.build(context, term, true);
+          allTermStats.add(searcher.termStatistics(term, termContext));
         }
       }
-      stats = similarity.computeStats(searcher, field, getBoost(), allTerms.toArray(new TermContext[allTerms.size()]));
+      stats = similarity.computeStats(searcher.collectionStatistics(field), 
+          getBoost(), allTermStats.toArray(new TermStatistics[allTermStats.size()]));
     }
 
     @Override

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/PhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/PhraseQuery.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/PhraseQuery.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/PhraseQuery.java Sat Nov  5 12:46:53 2011
@@ -190,9 +190,13 @@ public class PhraseQuery extends Query {
       this.similarity = searcher.getSimilarityProvider().get(field);
       final ReaderContext context = searcher.getTopReaderContext();
       states = new TermContext[terms.size()];
-      for (int i = 0; i < terms.size(); i++)
-        states[i] = TermContext.build(context, terms.get(i), true);
-      stats = similarity.computeStats(searcher, field, getBoost(), states);
+      TermStatistics termStats[] = new TermStatistics[terms.size()];
+      for (int i = 0; i < terms.size(); i++) {
+        final Term term = terms.get(i);
+        states[i] = TermContext.build(context, term, true);
+        termStats[i] = searcher.termStatistics(term, states[i]);
+      }
+      stats = similarity.computeStats(searcher.collectionStatistics(field), getBoost(), termStats);
     }
 
     @Override

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/TermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/TermQuery.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/TermQuery.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/TermQuery.java Sat Nov  5 12:46:53 2011
@@ -54,7 +54,10 @@ public class TermQuery extends Query {
       assert termStates != null : "TermContext must not be null";
       this.termStates = termStates;
       this.similarity = searcher.getSimilarityProvider().get(term.field());
-      this.stats = similarity.computeStats(searcher, term.field(), getBoost(), termStates);
+      this.stats = similarity.computeStats(
+          searcher.collectionStatistics(term.field()), 
+          getBoost(), 
+          searcher.termStatistics(term, termStates));
     }
 
     @Override

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/similarities/BM25Similarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/similarities/BM25Similarity.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/similarities/BM25Similarity.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/similarities/BM25Similarity.java Sat Nov  5 12:46:53 2011
@@ -20,14 +20,12 @@ package org.apache.lucene.search.similar
 import java.io.IOException;
 
 import org.apache.lucene.index.FieldInvertState;
-import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
-import org.apache.lucene.index.Terms;
+import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.Explanation;
-import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.TermStatistics;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.SmallFloat;
-import org.apache.lucene.util.TermContext;
 
 /**
  * BM25 Similarity. Introduced in Stephen E. Robertson, Steve Walker,
@@ -75,15 +73,13 @@ public class BM25Similarity extends Simi
   /** The default implementation computes the average as <code>sumTotalTermFreq / maxDoc</code>,
    * or returns <code>1</code> if the index does not store sumTotalTermFreq (Lucene 3.x indexes
    * or any field that omits frequency information). */
-  protected float avgFieldLength(IndexSearcher searcher, String field) throws IOException {
-    Terms terms = MultiFields.getTerms(searcher.getIndexReader(), field);
-    if (terms == null) {
-      // field does not exist;
-      return 1f;
-    }
-    long sumTotalTermFreq = terms.getSumTotalTermFreq();
-    long maxdoc = searcher.maxDoc();
-    return sumTotalTermFreq == -1 ? 1f : (float) (sumTotalTermFreq / (double) maxdoc);
+  protected float avgFieldLength(CollectionStatistics collectionStats) {
+    final long sumTotalTermFreq = collectionStats.sumTotalTermFreq();
+    if (sumTotalTermFreq <= 0) {
+      return 1f;       // field does not exist, or stat is unsupported
+    } else {
+      return (float) (sumTotalTermFreq / (double) collectionStats.maxDoc());
+    }
   }
   
   /** The default implementation encodes <code>boost / sqrt(length)</code>
@@ -131,19 +127,19 @@ public class BM25Similarity extends Simi
     return encodeNormValue(state.getBoost(), numTerms);
   }
 
-  public Explanation idfExplain(TermContext stats, final IndexSearcher searcher) throws IOException {
-    final int df = stats.docFreq();
-    final int max = searcher.maxDoc();
+  public Explanation idfExplain(CollectionStatistics collectionStats, TermStatistics termStats) {
+    final int df = termStats.docFreq();
+    final int max = collectionStats.maxDoc();
     final float idf = idf(df, max);
     return new Explanation(idf, "idf(docFreq=" + df + ", maxDocs=" + max + ")");
   }
 
-  public Explanation idfExplain(final TermContext stats[], IndexSearcher searcher) throws IOException {
-    final int max = searcher.maxDoc();
+  public Explanation idfExplain(CollectionStatistics collectionStats, TermStatistics termStats[]) {
+    final int max = collectionStats.maxDoc();
     float idf = 0.0f;
     final Explanation exp = new Explanation();
     exp.setDescription("idf(), sum of:");
-    for (final TermContext stat : stats ) {
+    for (final TermStatistics stat : termStats ) {
       final int df = stat.docFreq();
       final float termIdf = idf(df, max);
       exp.addDetail(new Explanation(termIdf, "idf(docFreq=" + df + ", maxDocs=" + max + ")"));
@@ -154,10 +150,10 @@ public class BM25Similarity extends Simi
   }
 
   @Override
-  public final Stats computeStats(IndexSearcher searcher, String fieldName, float queryBoost, TermContext... termStats) throws IOException {
-    Explanation idf = termStats.length == 1 ? idfExplain(termStats[0], searcher) : idfExplain(termStats, searcher);
+  public final Stats computeStats(CollectionStatistics collectionStats, float queryBoost, TermStatistics... termStats) {
+    Explanation idf = termStats.length == 1 ? idfExplain(collectionStats, termStats[0]) : idfExplain(collectionStats, termStats);
 
-    float avgdl = avgFieldLength(searcher, fieldName);
+    float avgdl = avgFieldLength(collectionStats);
 
     // compute freq-independent part of bm25 equation across all norm values
     float cache[] = new float[256];

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/similarities/LMSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/similarities/LMSimilarity.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/similarities/LMSimilarity.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/similarities/LMSimilarity.java Sat Nov  5 12:46:53 2011
@@ -17,11 +17,9 @@ package org.apache.lucene.search.similar
  * limitations under the License.
  */
 
-import java.io.IOException;
-
+import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.Explanation;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.util.TermContext;
+import org.apache.lucene.search.TermStatistics;
 
 /**
  * Abstract superclass for language modeling Similarities. The following inner
@@ -62,8 +60,8 @@ public abstract class LMSimilarity exten
    * usual statistics.
    */
   @Override
-  protected void fillBasicStats(BasicStats stats, IndexSearcher searcher, String fieldName, TermContext termContext) throws IOException {
-    super.fillBasicStats(stats, searcher, fieldName, termContext);
+  protected void fillBasicStats(BasicStats stats, CollectionStatistics collectionStats, TermStatistics termStats) {
+    super.fillBasicStats(stats, collectionStats, termStats);
     LMStats lmStats = (LMStats) stats;
     lmStats.setCollectionProbability(collectionModel.computeProbability(stats));
   }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java Sat Nov  5 12:46:53 2011
@@ -21,10 +21,10 @@ import java.io.IOException;
 
 import org.apache.lucene.index.FieldInvertState;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.Explanation;
-import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.TermStatistics;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.TermContext;
 
 /**
  * Implements the CombSUM method for combining evidence from multiple
@@ -45,10 +45,10 @@ public class MultiSimilarity extends Sim
   }
 
   @Override
-  public Stats computeStats(IndexSearcher searcher, String fieldName, float queryBoost, TermContext... termContexts) throws IOException {
+  public Stats computeStats(CollectionStatistics collectionStats, float queryBoost, TermStatistics... termStats) {
     Stats subStats[] = new Stats[sims.length];
     for (int i = 0; i < subStats.length; i++) {
-      subStats[i] = sims[i].computeStats(searcher, fieldName, queryBoost, termContexts);
+      subStats[i] = sims[i].computeStats(collectionStats, queryBoost, termStats);
     }
     return new MultiStats(subStats);
   }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/similarities/Similarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/similarities/Similarity.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/similarities/Similarity.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/similarities/Similarity.java Sat Nov  5 12:46:53 2011
@@ -26,11 +26,13 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Terms; // javadoc
 import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.PhraseQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TermStatistics;
 import org.apache.lucene.search.spans.SpanQuery; // javadoc
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.SmallFloat; // javadoc
@@ -81,10 +83,10 @@ import org.apache.lucene.util.TermContex
  * <a name="querytime"/>
  * At query-time, Queries interact with the Similarity via these steps:
  * <ol>
- *   <li>The {@link #computeStats(IndexSearcher, String, float, TermContext...)} method is called a single time,
+ *   <li>The {@link #computeStats(CollectionStatistics, float, TermStatistics...)} method is called a single time,
  *       allowing the implementation to compute any statistics (such as IDF, average document length, etc)
- *       across <i>the entire collection</i>. The {@link TermContext}s passed in are already positioned
- *       to the terms involved with the raw statistics involved, so a Similarity can freely use any combination
+ *       across <i>the entire collection</i>. The {@link TermStatistics} passed in already contain
+ *       the raw statistics involved, so a Similarity can freely use any combination
  *       of term statistics without causing any additional I/O. Lucene makes no assumption about what is 
  *       stored in the returned {@link Similarity.Stats} object.
  *   <li>The query normalization process occurs a single time: {@link Similarity.Stats#getValueForNormalization()}
@@ -128,7 +130,7 @@ public abstract class Similarity {
   /**
    * Compute any collection-level stats (e.g. IDF, average document length, etc) needed for scoring a query.
    */
-  public abstract Stats computeStats(IndexSearcher searcher, String fieldName, float queryBoost, TermContext... termContexts) throws IOException;
+  public abstract Stats computeStats(CollectionStatistics collectionStats, float queryBoost, TermStatistics... termStats);
   
   /**
    * returns a new {@link Similarity.ExactDocScorer}.

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/similarities/SimilarityBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/similarities/SimilarityBase.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/similarities/SimilarityBase.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/similarities/SimilarityBase.java Sat Nov  5 12:46:53 2011
@@ -20,15 +20,12 @@ package org.apache.lucene.search.similar
 import java.io.IOException;
 
 import org.apache.lucene.index.FieldInvertState;
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
-import org.apache.lucene.index.MultiFields;
-import org.apache.lucene.index.Terms;
+import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.Explanation;
-import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.TermStatistics;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.SmallFloat;
-import org.apache.lucene.util.TermContext;
 
 /**
  * A subclass of {@code Similarity} that provides a simplified API for its
@@ -71,12 +68,11 @@ public abstract class SimilarityBase ext
   }
   
   @Override
-  public final Stats computeStats(IndexSearcher searcher, String fieldName,
-      float queryBoost, TermContext... termContexts) throws IOException {
-    BasicStats stats[] = new BasicStats[termContexts.length];
-    for (int i = 0; i < termContexts.length; i++) {
+  public final Stats computeStats(CollectionStatistics collectionStats, float queryBoost, TermStatistics... termStats) {
+    BasicStats stats[] = new BasicStats[termStats.length];
+    for (int i = 0; i < termStats.length; i++) {
       stats[i] = newStats(queryBoost);
-      fillBasicStats(stats[i], searcher, fieldName, termContexts[i]);
+      fillBasicStats(stats[i], collectionStats, termStats[i]);
     }
     return stats.length == 1 ? stats[0] : new MultiSimilarity.MultiStats(stats);
   }
@@ -88,13 +84,11 @@ public abstract class SimilarityBase ext
   
   /** Fills all member fields defined in {@code BasicStats} in {@code stats}. 
    *  Subclasses can override this method to fill additional stats. */
-  protected void fillBasicStats(BasicStats stats, IndexSearcher searcher,
-      String fieldName, TermContext termContext) throws IOException {
-    IndexReader reader = searcher.getIndexReader();
-    int numberOfDocuments = reader.maxDoc();
+  protected void fillBasicStats(BasicStats stats, CollectionStatistics collectionStats, TermStatistics termStats) {
+    int numberOfDocuments = collectionStats.maxDoc();
     
-    int docFreq = termContext.docFreq();
-    long totalTermFreq = termContext.totalTermFreq();
+    int docFreq = termStats.docFreq();
+    long totalTermFreq = termStats.totalTermFreq();
 
     // codec does not supply totalTermFreq: substitute docFreq
     if (totalTermFreq == -1) {
@@ -103,25 +97,19 @@ public abstract class SimilarityBase ext
 
     final long numberOfFieldTokens;
     final float avgFieldLength;
-    
-    Terms terms = MultiFields.getTerms(searcher.getIndexReader(), fieldName);
-    if (terms == null) {
-      // field does not exist;
-      numberOfFieldTokens = 0;
-      avgFieldLength = 1;
-    } else {
-      long sumTotalTermFreq = terms.getSumTotalTermFreq();
 
+    long sumTotalTermFreq = collectionStats.sumTotalTermFreq();
+
+    if (sumTotalTermFreq <= 0) {
+      // field does not exist;
       // We have to provide something if codec doesnt supply these measures,
       // or if someone omitted frequencies for the field... negative values cause
       // NaN/Inf for some scorers.
-      if (sumTotalTermFreq == -1) {
-        numberOfFieldTokens = docFreq;
-        avgFieldLength = 1;
-      } else {
-        numberOfFieldTokens = sumTotalTermFreq;
-        avgFieldLength = (float)numberOfFieldTokens / numberOfDocuments;
-      }
+      numberOfFieldTokens = docFreq;
+      avgFieldLength = 1;
+    } else {
+      numberOfFieldTokens = sumTotalTermFreq;
+      avgFieldLength = (float)numberOfFieldTokens / numberOfDocuments;
     }
  
     // TODO: add sumDocFreq for field (numberOfFieldPostings)

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java Sat Nov  5 12:46:53 2011
@@ -22,9 +22,11 @@ import java.io.IOException;
 
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.PhraseQuery;
+import org.apache.lucene.search.TermStatistics;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.TermContext;
 import org.apache.lucene.util.SmallFloat;
@@ -569,21 +571,21 @@ public abstract class TFIDFSimilarity ex
    * idf(docFreq, searcher.maxDoc());
    * </pre>
    * 
-   * Note that {@link IndexSearcher#maxDoc()} is used instead of
+   * Note that {@link CollectionStatistics#maxDoc()} is used instead of
    * {@link org.apache.lucene.index.IndexReader#numDocs() IndexReader#numDocs()} because also 
-   * {@link IndexSearcher#docFreq(Term)} is used, and when the latter 
-   * is inaccurate, so is {@link IndexSearcher#maxDoc()}, and in the same direction.
-   * In addition, {@link IndexSearcher#maxDoc()} is more efficient to compute
+   * {@link TermStatistics#docFreq()} is used, and when the latter 
+   * is inaccurate, so is {@link CollectionStatistics#maxDoc()}, and in the same direction.
+   * In addition, {@link CollectionStatistics#maxDoc()} is more efficient to compute
    *   
-   * @param stats statistics of the term in question
-   * @param searcher the document collection being searched
+   * @param collectionStats collection-level statistics
+   * @param termStats term-level statistics for the term
    * @return an Explain object that includes both an idf score factor 
              and an explanation for the term.
    * @throws IOException
    */
-  public Explanation idfExplain(TermContext stats, final IndexSearcher searcher) throws IOException {
-    final int df = stats.docFreq();
-    final int max = searcher.maxDoc();
+  public Explanation idfExplain(CollectionStatistics collectionStats, TermStatistics termStats) {
+    final int df = termStats.docFreq();
+    final int max = collectionStats.maxDoc();
     final float idf = idf(df, max);
     return new Explanation(idf, "idf(docFreq=" + df + ", maxDocs=" + max + ")");
   }
@@ -595,19 +597,19 @@ public abstract class TFIDFSimilarity ex
    * The default implementation sums the idf factor for
    * each term in the phrase.
    * 
-   * @param stats statistics of the terms in the phrase
-   * @param searcher the document collection being searched
+   * @param collectionStats collection-level statistics
+   * @param termStats term-level statistics for the terms in the phrase
    * @return an Explain object that includes both an idf 
    *         score factor for the phrase and an explanation 
    *         for each term.
    * @throws IOException
    */
-  public Explanation idfExplain(final TermContext stats[], IndexSearcher searcher) throws IOException {
-    final int max = searcher.maxDoc();
+  public Explanation idfExplain(CollectionStatistics collectionStats, TermStatistics termStats[]) {
+    final int max = collectionStats.maxDoc();
     float idf = 0.0f;
     final Explanation exp = new Explanation();
     exp.setDescription("idf(), sum of:");
-    for (final TermContext stat : stats ) {
+    for (final TermStatistics stat : termStats ) {
       final int df = stat.docFreq();
       final float termIdf = idf(df, max);
       exp.addDetail(new Explanation(termIdf, "idf(docFreq=" + df + ", maxDocs=" + max + ")"));
@@ -693,11 +695,10 @@ public abstract class TFIDFSimilarity ex
   public abstract float scorePayload(int doc, int start, int end, BytesRef payload);
 
   @Override
-  public final Stats computeStats(IndexSearcher searcher, String fieldName, float queryBoost,
-      TermContext... termContexts) throws IOException {
-    final Explanation idf = termContexts.length == 1
-    ? idfExplain(termContexts[0], searcher)
-    : idfExplain(termContexts, searcher);
+  public final Stats computeStats(CollectionStatistics collectionStats, float queryBoost, TermStatistics... termStats) {
+    final Explanation idf = termStats.length == 1
+    ? idfExplain(collectionStats, termStats[0])
+    : idfExplain(collectionStats, termStats);
     return new IDFStats(idf, queryBoost);
   }
 

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java Sat Nov  5 12:46:53 2011
@@ -48,10 +48,17 @@ public class SpanWeight extends Weight {
     query.extractTerms(terms);
     final ReaderContext context = searcher.getTopReaderContext();
     final TermContext states[] = new TermContext[terms.size()];
+    final TermStatistics termStats[] = new TermStatistics[terms.size()];
     int i = 0;
-    for (Term term : terms)
-      states[i++] = TermContext.build(context, term, true);
-    stats = similarity.computeStats(searcher, query.getField(), query.getBoost(), states);
+    for (Term term : terms) {
+      states[i] = TermContext.build(context, term, true);
+      termStats[i] = searcher.termStatistics(term, states[i]);
+      i++;
+    }
+    stats = similarity.computeStats(
+        searcher.collectionStatistics(query.getField()), 
+        query.getBoost(), 
+        termStats);
   }
 
   @Override

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/BufferedIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/BufferedIndexInput.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/BufferedIndexInput.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/BufferedIndexInput.java Sat Nov  5 12:46:53 2011
@@ -51,14 +51,17 @@ public abstract class BufferedIndexInput
     return buffer[bufferPosition++];
   }
 
-  public BufferedIndexInput() {}
-  
-  public BufferedIndexInput(IOContext context) {
-    this(bufferSize(context));
+  public BufferedIndexInput(String resourceDesc) {
+    this(resourceDesc, BUFFER_SIZE);
+  }
+
+  public BufferedIndexInput(String resourceDesc, IOContext context) {
+    this(resourceDesc, bufferSize(context));
   }
 
   /** Inits BufferedIndexInput with a specific bufferSize */
-  public BufferedIndexInput(int bufferSize) {
+  public BufferedIndexInput(String resourceDesc, int bufferSize) {
+    super(resourceDesc);
     checkBufferSize(bufferSize);
     this.bufferSize = bufferSize;
   }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/ChecksumIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/ChecksumIndexInput.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/ChecksumIndexInput.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/ChecksumIndexInput.java Sat Nov  5 12:46:53 2011
@@ -31,6 +31,7 @@ public class ChecksumIndexInput extends 
   Checksum digest;
 
   public ChecksumIndexInput(IndexInput main) {
+    super("ChecksumIndexInput(" + main + ")");
     this.main = main;
     digest = new CRC32();
   }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java Sat Nov  5 12:46:53 2011
@@ -106,7 +106,9 @@ public final class CompoundFileDirectory
               numEntries);
           for (int i = 0; i < numEntries; i++) {
             final FileEntry fileEntry = new FileEntry();
-            mapping.put(input.readString(), fileEntry);
+            final String id = input.readString();
+            assert !mapping.containsKey(id): "id=" + id + " was written multiple times in the CFS";
+            mapping.put(id, fileEntry);
             fileEntry.offset = input.readLong();
             fileEntry.length = input.readLong();
           }
@@ -137,7 +139,7 @@ public final class CompoundFileDirectory
     if (firstInt < CompoundFileWriter.FORMAT_PRE_VERSION) {
       if (firstInt < CompoundFileWriter.FORMAT_CURRENT) {
         throw new CorruptIndexException("Incompatible format version: "
-            + firstInt + " expected " + CompoundFileWriter.FORMAT_CURRENT);
+            + firstInt + " expected " + CompoundFileWriter.FORMAT_CURRENT + " (resource: " + stream + ")");
       }
       // It's a post-3.1 index, read the count.
       count = stream.readVInt();
@@ -153,7 +155,7 @@ public final class CompoundFileDirectory
     for (int i=0; i<count; i++) {
       long offset = stream.readLong();
       if (offset < 0 || offset > streamLength) {
-        throw new CorruptIndexException("Invalid CFS entry offset: " + offset);
+        throw new CorruptIndexException("Invalid CFS entry offset: " + offset + " (resource: " + stream + ")");
       }
       String id = stream.readString();
       
@@ -170,6 +172,9 @@ public final class CompoundFileDirectory
       
       entry = new FileEntry();
       entry.offset = offset;
+
+      assert !entries.containsKey(id);
+
       entries.put(id, entry);
     }
     
@@ -213,7 +218,7 @@ public final class CompoundFileDirectory
     if (entry == null) {
       throw new IOException("No sub-file with id " + id + " found (fileName=" + name + " files: " + entries.keySet() + ")");
     }
-    return handle.openSlice(entry.offset, entry.length);
+    return handle.openSlice(name, entry.offset, entry.length);
   }
   
   /** Returns an array of strings, one for each file in the directory. */
@@ -271,7 +276,7 @@ public final class CompoundFileDirectory
   public long fileLength(String name) throws IOException {
     ensureOpen();
     if (this.writer != null) {
-      return writer.fileLenght(name);
+      return writer.fileLength(name);
     }
     FileEntry e = entries.get(IndexFileNames.stripSegmentName(name));
     if (e == null)
@@ -313,14 +318,19 @@ public final class CompoundFileDirectory
       }
       
       @Override
-      public IndexInput openSlice(long offset, long length) throws IOException {
-        return handle.openSlice(entry.offset + offset, length);
+      public IndexInput openSlice(String sliceDescription, long offset, long length) throws IOException {
+        return handle.openSlice(sliceDescription, entry.offset + offset, length);
       }
 
       @Override
       public IndexInput openFullSlice() throws IOException {
-        return openSlice(0, entry.length);
+        return openSlice("full-slice", 0, entry.length);
       }
     };
   }
+
+  @Override
+  public String toString() {
+    return "CompoundFileDirectory(file=\"" + fileName + "\" in dir=" + directory + ")";
+  }
 }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/CompoundFileWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/CompoundFileWriter.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/CompoundFileWriter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/CompoundFileWriter.java Sat Nov  5 12:46:53 2011
@@ -22,9 +22,11 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.Map;
 import java.util.Queue;
+import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.lucene.index.IndexFileNames;
@@ -85,6 +87,7 @@ final class CompoundFileWriter implement
 
   private final Directory directory;
   private final Map<String, FileEntry> entries = new HashMap<String, FileEntry>();
+  private final Set<String> seenIDs = new HashSet<String>();
   // all entries that are written to a sep. file but not yet moved into CFS
   private final Queue<FileEntry> pendingEntries = new LinkedList<FileEntry>();
   private boolean closed = false;
@@ -238,6 +241,9 @@ final class CompoundFileWriter implement
       final FileEntry entry = new FileEntry();
       entry.file = name;
       entries.put(name, entry);
+      final String id = IndexFileNames.stripSegmentName(name);
+      assert !seenIDs.contains(id): "file=\"" + name + "\" maps to id=\"" + id + "\", which was already written";
+      seenIDs.add(id);
       final DirectCFSIndexOutput out;
       if (outputTaken.compareAndSet(false, true)) {
         out = new DirectCFSIndexOutput(dataOut, entry, false);
@@ -284,7 +290,7 @@ final class CompoundFileWriter implement
     }
   }
 
-  long fileLenght(String name) throws IOException {
+  long fileLength(String name) throws IOException {
     FileEntry fileEntry = entries.get(name);
     if (fileEntry == null) {
       throw new FileNotFoundException(name + " does not exist");

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/Directory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/Directory.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/Directory.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/Directory.java Sat Nov  5 12:46:53 2011
@@ -225,8 +225,8 @@ public abstract class Directory implemen
     return new IndexInputSlicer() {
       private final IndexInput base = Directory.this.openInput(name, context);
       @Override
-      public IndexInput openSlice(long offset, long length) {
-        return new SlicedIndexInput(base, offset, length);
+      public IndexInput openSlice(String sliceDescription, long offset, long length) {
+        return new SlicedIndexInput("SlicedIndexInput(" + sliceDescription + " in " + base + ")", base, offset, length);
       }
       @Override
       public void close() throws IOException {
@@ -258,7 +258,7 @@ public abstract class Directory implemen
     /**
      * Returns an {@link IndexInput} slice starting at the given offset with the given length.
      */
-    public abstract IndexInput openSlice(long offset, long length) throws IOException;
+    public abstract IndexInput openSlice(String sliceDescription, long offset, long length) throws IOException;
 
     /**
      * Returns an {@link IndexInput} slice starting at offset <i>0</i> with a
@@ -275,12 +275,12 @@ public abstract class Directory implemen
     long fileOffset;
     long length;
     
-    SlicedIndexInput(final IndexInput base, final long fileOffset, final long length) {
-      this(base, fileOffset, length, BufferedIndexInput.BUFFER_SIZE);
+    SlicedIndexInput(final String sliceDescription, final IndexInput base, final long fileOffset, final long length) {
+      this(sliceDescription, base, fileOffset, length, BufferedIndexInput.BUFFER_SIZE);
     }
     
-    SlicedIndexInput(final IndexInput base, final long fileOffset, final long length, int readBufferSize) {
-      super(readBufferSize);
+    SlicedIndexInput(final String sliceDescription, final IndexInput base, final long fileOffset, final long length, int readBufferSize) {
+      super("SlicedIndexInput(" + sliceDescription + " in " + base + " slice=" + fileOffset + ":" + (fileOffset+length) + ")", readBufferSize);
       this.base = (IndexInput) base.clone();
       this.fileOffset = fileOffset;
       this.length = length;

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/IndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/IndexInput.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/IndexInput.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/IndexInput.java Sat Nov  5 12:46:53 2011
@@ -26,6 +26,18 @@ import java.io.IOException;
  */
 public abstract class IndexInput extends DataInput implements Cloneable,Closeable {
 
+  private final String resourceDescription;
+
+  /** resourceDescription should be a non-null, opaque string
+   *  describing this resource; it's returned from
+   *  {@link #toString}. */
+  protected IndexInput(String resourceDescription) {
+    if (resourceDescription == null) {
+      throw new IllegalArgumentException("resourceDescription must not be null");
+    }
+    this.resourceDescription = resourceDescription;
+  }
+
   /** Closes the stream to further operations. */
   public abstract void close() throws IOException;
 
@@ -66,5 +78,9 @@ public abstract class IndexInput extends
       numBytes -= toCopy;
     }
   }
-  
+
+  @Override
+  public String toString() {
+    return resourceDescription;
+  }
 }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/MMapDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/MMapDirectory.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/MMapDirectory.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/MMapDirectory.java Sat Nov  5 12:46:53 2011
@@ -213,7 +213,7 @@ public class MMapDirectory extends FSDir
     File f = new File(getDirectory(), name);
     RandomAccessFile raf = new RandomAccessFile(f, "r");
     try {
-      return new MMapIndexInput(raf, 0, raf.length(), chunkSizePower);
+      return new MMapIndexInput("MMapIndexInput(path=\"" + f + "\")", raf, 0, raf.length(), chunkSizePower);
     } finally {
       raf.close();
     }
@@ -221,7 +221,7 @@ public class MMapDirectory extends FSDir
   
   public IndexInputSlicer createSlicer(final String name, final IOContext context) throws IOException {
     ensureOpen();
-    File f = new File(getDirectory(), name);
+    final File f = new File(getDirectory(), name);
     final RandomAccessFile raf = new RandomAccessFile(f, "r");
     return new IndexInputSlicer() {
       @Override
@@ -230,13 +230,13 @@ public class MMapDirectory extends FSDir
       }
 
       @Override
-      public IndexInput openSlice(long offset, long length) throws IOException {
-        return new MMapIndexInput(raf, offset, length, chunkSizePower);
+      public IndexInput openSlice(String sliceDescription, long offset, long length) throws IOException {
+        return new MMapIndexInput("MMapIndexInput(" + sliceDescription + " in path=\"" + f + "\" slice=" + offset + ":" + (offset+length) + ")", raf, offset, length, chunkSizePower);
       }
 
       @Override
       public IndexInput openFullSlice() throws IOException {
-        return openSlice(0, raf.length());
+        return openSlice("full-slice", 0, raf.length());
       }
     };
   }
@@ -256,8 +256,9 @@ public class MMapDirectory extends FSDir
     private ByteBuffer curBuf; // redundant for speed: buffers[curBufIndex]
   
     private boolean isClone = false;
-    
-    MMapIndexInput(RandomAccessFile raf, long offset, long length, int chunkSizePower) throws IOException {
+
+    MMapIndexInput(String resourceDescription, RandomAccessFile raf, long offset, long length, int chunkSizePower) throws IOException {
+      super(resourceDescription);
       this.length = length;
       this.chunkSizePower = chunkSizePower;
       this.chunkSize = 1L << chunkSizePower;
@@ -296,8 +297,9 @@ public class MMapDirectory extends FSDir
       } catch (BufferUnderflowException e) {
         do {
           curBufIndex++;
-          if (curBufIndex >= buffers.length)
-            throw new IOException("read past EOF");
+          if (curBufIndex >= buffers.length) {
+            throw new IOException("read past EOF: " + this);
+          }
           curBuf = buffers[curBufIndex];
           curBuf.position(0);
         } while (!curBuf.hasRemaining());
@@ -316,8 +318,9 @@ public class MMapDirectory extends FSDir
           len -= curAvail;
           offset += curAvail;
           curBufIndex++;
-          if (curBufIndex >= buffers.length)
-            throw new IOException("read past EOF");
+          if (curBufIndex >= buffers.length) {
+            throw new IOException("read past EOF: " + this);
+          }
           curBuf = buffers[curBufIndex];
           curBuf.position(0);
           curAvail = curBuf.remaining();
@@ -369,13 +372,15 @@ public class MMapDirectory extends FSDir
         this.curBufIndex = bi;
         this.curBuf = b;
       } catch (ArrayIndexOutOfBoundsException aioobe) {
-        if (pos < 0L)
-          throw new IllegalArgumentException("Seeking to negative position");
+        if (pos < 0L) {
+          throw new IllegalArgumentException("Seeking to negative position: " + this);
+        }
         throw new IOException("seek past EOF");
       } catch (IllegalArgumentException iae) {
-        if (pos < 0L)
-          throw new IllegalArgumentException("Seeking to negative position");
-        throw new IOException("seek past EOF");
+        if (pos < 0L) {
+          throw new IllegalArgumentException("Seeking to negative position: " + this);
+        }
+        throw new IOException("seek past EOF: " + this);
       }
     }
   
@@ -386,8 +391,9 @@ public class MMapDirectory extends FSDir
   
     @Override
     public Object clone() {
-      if (buffers == null)
-        throw new AlreadyClosedException("MMapIndexInput already closed");
+      if (buffers == null) {
+        throw new AlreadyClosedException("MMapIndexInput already closed: " + this);
+      }
       final MMapIndexInput clone = (MMapIndexInput)super.clone();
       clone.isClone = true;
       clone.buffers = new ByteBuffer[buffers.length];
@@ -399,7 +405,7 @@ public class MMapDirectory extends FSDir
       try {
         clone.seek(getFilePointer());
       } catch(IOException ioe) {
-        throw new RuntimeException("Should never happen", ioe);
+        throw new RuntimeException("Should never happen: " + this, ioe);
       }
       return clone;
     }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java Sat Nov  5 12:46:53 2011
@@ -83,8 +83,8 @@ public class NIOFSDirectory extends FSDi
   public IndexInputSlicer createSlicer(final String name,
       final IOContext context) throws IOException {
     ensureOpen();
-    final File file = new File(getDirectory(), name);
-    final Descriptor descriptor = new Descriptor(file, "r");
+    final File path = new File(getDirectory(), name);
+    final Descriptor descriptor = new Descriptor(path, "r");
     return new Directory.IndexInputSlicer() {
 
       @Override
@@ -93,14 +93,14 @@ public class NIOFSDirectory extends FSDi
       }
 
       @Override
-      public IndexInput openSlice(long offset, long length) throws IOException {
-        return new NIOFSIndexInput(descriptor, descriptor.getChannel(), offset,
+      public IndexInput openSlice(String sliceDescription, long offset, long length) throws IOException {
+        return new NIOFSIndexInput(sliceDescription, path, descriptor, descriptor.getChannel(), offset,
             length, BufferedIndexInput.bufferSize(context), getReadChunkSize());
       }
 
       @Override
       public IndexInput openFullSlice() throws IOException {
-        return openSlice(0, descriptor.length);
+        return openSlice("full-slice", 0, descriptor.length);
       }
     };
   }
@@ -115,12 +115,12 @@ public class NIOFSDirectory extends FSDi
     final FileChannel channel;
 
     public NIOFSIndexInput(File path, IOContext context, int chunkSize) throws IOException {
-      super(path, context, chunkSize);
+      super("NIOFSIndexInput(path=\"" + path + "\")", path, context, chunkSize);
       channel = file.getChannel();
     }
     
-    public NIOFSIndexInput(Descriptor file, FileChannel fc, long off, long length, int bufferSize, int chunkSize) throws IOException {
-      super(file, off, length, bufferSize, chunkSize);
+    public NIOFSIndexInput(String sliceDescription, File path, Descriptor file, FileChannel fc, long off, long length, int bufferSize, int chunkSize) throws IOException {
+      super("NIOFSIndexInput(" + sliceDescription + " in path=\"" + path + "\" slice=" + off + ":" + (off+length) + ")", file, off, length, bufferSize, chunkSize);
       channel = fc;
       isClone = true;
     }
@@ -181,7 +181,7 @@ public class NIOFSDirectory extends FSDi
       long pos = getFilePointer() + off;
       
       if (pos + len > end) {
-        throw new IOException("read past EOF");
+        throw new IOException("read past EOF: " + this);
       }
 
       try {
@@ -209,6 +209,8 @@ public class NIOFSDirectory extends FSDi
               + "with a value smaller than the current chunk size (" + chunkSize + ")");
         outOfMemoryError.initCause(e);
         throw outOfMemoryError;
+      } catch (IOException ioe) {
+        throw new IOException(ioe.getMessage() + ": " + this, ioe);
       }
     }
   }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/RAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/RAMDirectory.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/RAMDirectory.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/store/RAMDirectory.java Sat Nov  5 12:46:53 2011
@@ -183,7 +183,7 @@ public class RAMDirectory extends Direct
     if (file == null) {
       throw new FileNotFoundException(name);
     }
-    return new RAMInputStream(file);
+    return new RAMInputStream(name, file);
   }
 
   /** Closes the store to future operations, releasing associated memory. */