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 2011/05/14 15:51:59 UTC

svn commit: r1103112 [6/24] - in /lucene/dev/branches/flexscoring: ./ dev-tools/eclipse/ dev-tools/idea/.idea/ dev-tools/idea/lucene/contrib/ant/ dev-tools/idea/lucene/contrib/db/bdb-je/ dev-tools/idea/lucene/contrib/db/bdb/ dev-tools/idea/lucene/contr...

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldsReader.java?rev=1103112&r1=1103111&r2=1103112&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldsReader.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldsReader.java Sat May 14 13:51:35 2011
@@ -24,10 +24,11 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldSelector;
 import org.apache.lucene.document.FieldSelectorResult;
 import org.apache.lucene.document.Fieldable;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.document.NumericField;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.BufferedIndexInput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.CloseableThreadLocal;
 
 import java.io.IOException;
@@ -212,40 +213,39 @@ public final class FieldsReader implemen
 
     Document doc = new Document();
     int numFields = fieldsStream.readVInt();
-    for (int i = 0; i < numFields; i++) {
+    out: for (int i = 0; i < numFields; i++) {
       int fieldNumber = fieldsStream.readVInt();
       FieldInfo fi = fieldInfos.fieldInfo(fieldNumber);
       FieldSelectorResult acceptField = fieldSelector == null ? FieldSelectorResult.LOAD : fieldSelector.accept(fi.name);
       
-      byte bits = fieldsStream.readByte();
-      assert bits <= FieldsWriter.FIELD_IS_TOKENIZED + FieldsWriter.FIELD_IS_BINARY;
+      int bits = fieldsStream.readByte() & 0xFF;
+      assert bits <= (FieldsWriter.FIELD_IS_NUMERIC_MASK | FieldsWriter.FIELD_IS_TOKENIZED | FieldsWriter.FIELD_IS_BINARY): "bits=" + Integer.toHexString(bits);
 
       boolean tokenize = (bits & FieldsWriter.FIELD_IS_TOKENIZED) != 0;
       boolean binary = (bits & FieldsWriter.FIELD_IS_BINARY) != 0;
-      //TODO: Find an alternative approach here if this list continues to grow beyond the
-      //list of 5 or 6 currently here.  See Lucene 762 for discussion
-      if (acceptField.equals(FieldSelectorResult.LOAD)) {
-        addField(doc, fi, binary, tokenize);
-      }
-      else if (acceptField.equals(FieldSelectorResult.LOAD_AND_BREAK)){
-        addField(doc, fi, binary, tokenize);
-        break;//Get out of this loop
-      }
-      else if (acceptField.equals(FieldSelectorResult.LAZY_LOAD)) {
-        addFieldLazy(doc, fi, binary, tokenize, true);
-      }
-      else if (acceptField.equals(FieldSelectorResult.LATENT)) {
-        addFieldLazy(doc, fi, binary, tokenize, false);
-      }
-      else if (acceptField.equals(FieldSelectorResult.SIZE)){
-        skipField(addFieldSize(doc, fi, binary));
-      }
-      else if (acceptField.equals(FieldSelectorResult.SIZE_AND_BREAK)){
-        addFieldSize(doc, fi, binary);
-        break;
-      }
-      else {
-        skipField();
+      final int numeric = bits & FieldsWriter.FIELD_IS_NUMERIC_MASK;
+
+      switch (acceptField) {
+        case LOAD:
+          addField(doc, fi, binary, tokenize, numeric);
+          break;
+        case LOAD_AND_BREAK:
+          addField(doc, fi, binary, tokenize, numeric);
+          break out; //Get out of this loop
+        case LAZY_LOAD:
+          addFieldLazy(doc, fi, binary, tokenize, true, numeric);
+          break;
+        case LATENT:
+          addFieldLazy(doc, fi, binary, tokenize, false, numeric);
+          break;
+        case SIZE:
+          skipFieldBytes(addFieldSize(doc, fi, binary, numeric));
+          break;
+        case SIZE_AND_BREAK:
+          addFieldSize(doc, fi, binary, numeric);
+          break out; //Get out of this loop
+        default:
+          skipField(numeric);
       }
     }
 
@@ -282,72 +282,121 @@ public final class FieldsReader implemen
    * Skip the field.  We still have to read some of the information about the field, but can skip past the actual content.
    * This will have the most payoff on large fields.
    */
-  private void skipField() throws IOException {
-    skipField(fieldsStream.readVInt());
+  private void skipField(int numeric) throws IOException {
+    final int numBytes;
+    switch(numeric) {
+      case 0:
+        numBytes = fieldsStream.readVInt();
+        break;
+      case FieldsWriter.FIELD_IS_NUMERIC_INT:
+      case FieldsWriter.FIELD_IS_NUMERIC_FLOAT:
+        numBytes = 4;
+        break;
+      case FieldsWriter.FIELD_IS_NUMERIC_LONG:
+      case FieldsWriter.FIELD_IS_NUMERIC_DOUBLE:
+        numBytes = 8;
+        break;
+      default:
+        throw new FieldReaderException("Invalid numeric type: " + Integer.toHexString(numeric));
+    }
+    
+    skipFieldBytes(numBytes);
   }
   
-  private void skipField(int toRead) throws IOException {
+  private void skipFieldBytes(int toRead) throws IOException {
     fieldsStream.seek(fieldsStream.getFilePointer() + toRead);
   }
 
-  private void addFieldLazy(Document doc, FieldInfo fi, boolean binary, boolean tokenize, boolean cacheResult) throws IOException {
+  private NumericField loadNumericField(FieldInfo fi, int numeric) throws IOException {
+    assert numeric != 0;
+    switch(numeric) {
+      case FieldsWriter.FIELD_IS_NUMERIC_INT:
+        return new NumericField(fi.name, Field.Store.YES, fi.isIndexed).setIntValue(fieldsStream.readInt());
+      case FieldsWriter.FIELD_IS_NUMERIC_LONG:
+        return new NumericField(fi.name, Field.Store.YES, fi.isIndexed).setLongValue(fieldsStream.readLong());
+      case FieldsWriter.FIELD_IS_NUMERIC_FLOAT:
+        return new NumericField(fi.name, Field.Store.YES, fi.isIndexed).setFloatValue(Float.intBitsToFloat(fieldsStream.readInt()));
+      case FieldsWriter.FIELD_IS_NUMERIC_DOUBLE:
+        return new NumericField(fi.name, Field.Store.YES, fi.isIndexed).setDoubleValue(Double.longBitsToDouble(fieldsStream.readLong()));
+      default:
+        throw new FieldReaderException("Invalid numeric type: " + Integer.toHexString(numeric));
+    }
+  }
+
+  private void addFieldLazy(Document doc, FieldInfo fi, boolean binary, boolean tokenize, boolean cacheResult, int numeric) throws IOException {
+    final AbstractField f;
     if (binary) {
       int toRead = fieldsStream.readVInt();
       long pointer = fieldsStream.getFilePointer();
-      //was: doc.add(new Fieldable(fi.name, b, Fieldable.Store.YES));
-      doc.add(new LazyField(fi.name, Field.Store.YES, toRead, pointer, binary, cacheResult));
+      f = new LazyField(fi.name, Field.Store.YES, toRead, pointer, binary, cacheResult);
       //Need to move the pointer ahead by toRead positions
       fieldsStream.seek(pointer + toRead);
+    } else if (numeric != 0) {
+      f = loadNumericField(fi, numeric);
     } else {
       Field.Store store = Field.Store.YES;
       Field.Index index = Field.Index.toIndex(fi.isIndexed, tokenize);
       Field.TermVector termVector = Field.TermVector.toTermVector(fi.storeTermVector, fi.storeOffsetWithTermVector, fi.storePositionWithTermVector);
 
-      AbstractField f;
       int length = fieldsStream.readVInt();
       long pointer = fieldsStream.getFilePointer();
       //Skip ahead of where we are by the length of what is stored
       fieldsStream.seek(pointer+length);
       f = new LazyField(fi.name, store, index, termVector, length, pointer, binary, cacheResult);
-      f.setOmitNorms(fi.omitNorms);
-      f.setOmitTermFreqAndPositions(fi.omitTermFreqAndPositions);
-
-      doc.add(f);
     }
-
+    
+    f.setOmitNorms(fi.omitNorms);
+    f.setOmitTermFreqAndPositions(fi.omitTermFreqAndPositions);
+    doc.add(f);
   }
 
-  private void addField(Document doc, FieldInfo fi, boolean binary, boolean tokenize) throws CorruptIndexException, IOException {
+  private void addField(Document doc, FieldInfo fi, boolean binary, boolean tokenize, int numeric) throws CorruptIndexException, IOException {
+    final AbstractField f;
 
     if (binary) {
       int toRead = fieldsStream.readVInt();
       final byte[] b = new byte[toRead];
       fieldsStream.readBytes(b, 0, b.length);
-      doc.add(new Field(fi.name, b));
+      f = new Field(fi.name, b);
+    } else if (numeric != 0) {
+      f = loadNumericField(fi, numeric);
     } else {
-      Field.Store store = Field.Store.YES;
       Field.Index index = Field.Index.toIndex(fi.isIndexed, tokenize);
       Field.TermVector termVector = Field.TermVector.toTermVector(fi.storeTermVector, fi.storeOffsetWithTermVector, fi.storePositionWithTermVector);
-
-      AbstractField f;
       f = new Field(fi.name,     // name
-       false,
-              fieldsStream.readString(), // read value
-              store,
-              index,
-              termVector);
-      f.setOmitTermFreqAndPositions(fi.omitTermFreqAndPositions);
-      f.setOmitNorms(fi.omitNorms);
-
-      doc.add(f);
+        false,
+        fieldsStream.readString(), // read value
+        Field.Store.YES,
+        index,
+        termVector);
     }
+    
+    f.setOmitTermFreqAndPositions(fi.omitTermFreqAndPositions);
+    f.setOmitNorms(fi.omitNorms);
+    doc.add(f);
   }
   
   // Add the size of field as a byte[] containing the 4 bytes of the integer byte size (high order byte first; char = 2 bytes)
   // Read just the size -- caller must skip the field content to continue reading fields
   // Return the size in bytes or chars, depending on field type
-  private int addFieldSize(Document doc, FieldInfo fi, boolean binary) throws IOException {
-    int size = fieldsStream.readVInt(), bytesize = binary ? size : 2*size;
+  private int addFieldSize(Document doc, FieldInfo fi, boolean binary, int numeric) throws IOException {
+    final int bytesize, size;
+    switch(numeric) {
+      case 0:
+        size = fieldsStream.readVInt();
+        bytesize = binary ? size : 2*size;
+        break;
+      case FieldsWriter.FIELD_IS_NUMERIC_INT:
+      case FieldsWriter.FIELD_IS_NUMERIC_FLOAT:
+        size = bytesize = 4;
+        break;
+      case FieldsWriter.FIELD_IS_NUMERIC_LONG:
+      case FieldsWriter.FIELD_IS_NUMERIC_DOUBLE:
+        size = bytesize = 8;
+        break;
+      default:
+        throw new FieldReaderException("Invalid numeric type: " + Integer.toHexString(numeric));
+    }
     byte[] sizebytes = new byte[4];
     sizebytes[0] = (byte) (bytesize>>>24);
     sizebytes[1] = (byte) (bytesize>>>16);
@@ -358,7 +407,7 @@ public final class FieldsReader implemen
   }
 
   /**
-   * A Lazy implementation of Fieldable that differs loading of fields until asked for, instead of when the Document is
+   * A Lazy implementation of Fieldable that defers loading of fields until asked for, instead of when the Document is
    * loaded.
    */
   private class LazyField extends AbstractField implements Fieldable {

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldsWriter.java?rev=1103112&r1=1103111&r2=1103112&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldsWriter.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldsWriter.java Sat May 14 13:51:35 2011
@@ -2,13 +2,13 @@ package org.apache.lucene.index;
 
 /**
  * Copyright 2004 The Apache Software Foundation
- * 
+ *
  * Licensed under the Apache License, Version 2.0 (the "License"); you may not
  * use this file except in compliance with the License. You may obtain a copy of
  * the License at
- * 
+ *
  * http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
  * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
@@ -21,24 +21,41 @@ import java.util.List;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Fieldable;
+import org.apache.lucene.document.NumericField;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.RAMOutputStream;
-import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.IOUtils;
 
 final class FieldsWriter {
-  static final byte FIELD_IS_TOKENIZED = 0x1;
-  static final byte FIELD_IS_BINARY = 0x2;
+  static final int FIELD_IS_TOKENIZED = 1 << 0;
+  static final int FIELD_IS_BINARY = 1 << 1;
+
+  // the old bit 1 << 2 was compressed, is now left out
+
+  private static final int _NUMERIC_BIT_SHIFT = 3;
+  static final int FIELD_IS_NUMERIC_MASK = 0x07 << _NUMERIC_BIT_SHIFT;
+
+  static final int FIELD_IS_NUMERIC_INT = 1 << _NUMERIC_BIT_SHIFT;
+  static final int FIELD_IS_NUMERIC_LONG = 2 << _NUMERIC_BIT_SHIFT;
+  static final int FIELD_IS_NUMERIC_FLOAT = 3 << _NUMERIC_BIT_SHIFT;
+  static final int FIELD_IS_NUMERIC_DOUBLE = 4 << _NUMERIC_BIT_SHIFT;
+  // currently unused: static final int FIELD_IS_NUMERIC_SHORT = 5 << _NUMERIC_BIT_SHIFT;
+  // currently unused: static final int FIELD_IS_NUMERIC_BYTE = 6 << _NUMERIC_BIT_SHIFT;
+
+  // the next possible bits are: 1 << 6; 1 << 7
   
   // Lucene 3.0: Removal of compressed fields
   static final int FORMAT_LUCENE_3_0_NO_COMPRESSED_FIELDS = 2;
 
+  // Lucene 3.2: NumericFields are stored in binary format
+  static final int FORMAT_LUCENE_3_2_NUMERIC_FIELDS = 3;
+
   // NOTE: if you introduce a new format, make it 1 higher
   // than the current one, and always change this if you
   // switch to a new format!
-  static final int FORMAT_CURRENT = FORMAT_LUCENE_3_0_NO_COMPRESSED_FIELDS;
-  
+  static final int FORMAT_CURRENT = FORMAT_LUCENE_3_2_NUMERIC_FIELDS;
+
   // when removing support for old versions, leave the last supported version here
   static final int FORMAT_MINIMUM = FORMAT_LUCENE_3_0_NO_COMPRESSED_FIELDS;
 
@@ -83,10 +100,9 @@ final class FieldsWriter {
   // and adds a new entry for this document into the index
   // stream.  This assumes the buffer was already written
   // in the correct fields format.
-  void flushDocument(int numStoredFields, RAMOutputStream buffer) throws IOException {
+  void startDocument(int numStoredFields) throws IOException {
     indexStream.writeLong(fieldsStream.getFilePointer());
     fieldsStream.writeVInt(numStoredFields);
-    buffer.writeTo(fieldsStream);
   }
 
   void skipDocument() throws IOException {
@@ -121,15 +137,28 @@ final class FieldsWriter {
     }
   }
 
-  final void writeField(FieldInfo fi, Fieldable field) throws IOException {
-    fieldsStream.writeVInt(fi.number);
-    byte bits = 0;
+  final void writeField(int fieldNumber, Fieldable field) throws IOException {
+    fieldsStream.writeVInt(fieldNumber);
+    int bits = 0;
     if (field.isTokenized())
-      bits |= FieldsWriter.FIELD_IS_TOKENIZED;
+      bits |= FIELD_IS_TOKENIZED;
     if (field.isBinary())
-      bits |= FieldsWriter.FIELD_IS_BINARY;
-
-    fieldsStream.writeByte(bits);
+      bits |= FIELD_IS_BINARY;
+    if (field instanceof NumericField) {
+      switch (((NumericField) field).getDataType()) {
+        case INT:
+          bits |= FIELD_IS_NUMERIC_INT; break;
+        case LONG:
+          bits |= FIELD_IS_NUMERIC_LONG; break;
+        case FLOAT:
+          bits |= FIELD_IS_NUMERIC_FLOAT; break;
+        case DOUBLE:
+          bits |= FIELD_IS_NUMERIC_DOUBLE; break;
+        default:
+          assert false : "Should never get here";
+      }
+    }
+    fieldsStream.writeByte((byte) bits);
 
     if (field.isBinary()) {
       final byte[] data;
@@ -141,8 +170,22 @@ final class FieldsWriter {
 
       fieldsStream.writeVInt(len);
       fieldsStream.writeBytes(data, offset, len);
-    }
-    else {
+    } else if (field instanceof NumericField) {
+      final NumericField nf = (NumericField) field;
+      final Number n = nf.getNumericValue();
+      switch (nf.getDataType()) {
+        case INT:
+          fieldsStream.writeInt(n.intValue()); break;
+        case LONG:
+          fieldsStream.writeLong(n.longValue()); break;
+        case FLOAT:
+          fieldsStream.writeInt(Float.floatToIntBits(n.floatValue())); break;
+        case DOUBLE:
+          fieldsStream.writeLong(Double.doubleToLongBits(n.doubleValue())); break;
+        default:
+          assert false : "Should never get here";
+      }
+    } else {
       fieldsStream.writeString(field.stringValue());
     }
   }
@@ -175,10 +218,9 @@ final class FieldsWriter {
     fieldsStream.writeVInt(storedCount);
 
 
-
     for (Fieldable field : fields) {
       if (field.isStored())
-        writeField(fieldInfos.fieldInfo(field.name()), field);
+        writeField(fieldInfos.fieldNumber(field.name()), field);
     }
   }
 }

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java?rev=1103112&r1=1103111&r2=1103112&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java Sat May 14 13:51:35 2011
@@ -19,55 +19,35 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.lucene.index.codecs.FieldsConsumer;
-import org.apache.lucene.index.codecs.PostingsConsumer;
-import org.apache.lucene.index.codecs.TermStats;
-import org.apache.lucene.index.codecs.TermsConsumer;
-import org.apache.lucene.util.BitVector;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CollectionUtil;
 
 final class FreqProxTermsWriter extends TermsHashConsumer {
 
   @Override
-  public TermsHashConsumerPerThread addThread(TermsHashPerThread perThread) {
-    return new FreqProxTermsWriterPerThread(perThread);
-  }
-
-  @Override
   void abort() {}
 
-  private int flushedDocCount;
-
   // TODO: would be nice to factor out more of this, eg the
   // FreqProxFieldMergeState, and code to visit all Fields
   // under the same FieldInfo together, up into TermsHash*.
   // Other writers would presumably share alot of this...
 
   @Override
-  public void flush(Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException {
+  public void flush(Map<FieldInfo, TermsHashConsumerPerField> fieldsToFlush, final SegmentWriteState state) throws IOException {
 
     // Gather all FieldData's that have postings, across all
     // ThreadStates
     List<FreqProxTermsWriterPerField> allFields = new ArrayList<FreqProxTermsWriterPerField>();
-    
-    flushedDocCount = state.numDocs;
-
-    for (Map.Entry<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> entry : threadsAndFields.entrySet()) {
-
-      Collection<TermsHashConsumerPerField> fields = entry.getValue();
-
 
-      for (final TermsHashConsumerPerField i : fields) {
-        final FreqProxTermsWriterPerField perField = (FreqProxTermsWriterPerField) i;
-        if (perField.termsHashPerField.bytesHash.size() > 0)
+    for (TermsHashConsumerPerField f : fieldsToFlush.values()) {
+        final FreqProxTermsWriterPerField perField = (FreqProxTermsWriterPerField) f;
+        if (perField.termsHashPerField.bytesHash.size() > 0) {
           allFields.add(perField);
-      }
+        }
     }
 
     final int numAllFields = allFields.size();
@@ -77,6 +57,8 @@ final class FreqProxTermsWriter extends 
 
     final FieldsConsumer consumer = state.segmentCodecs.codec().fieldsConsumer(state);
 
+    TermsHash termsHash = null;
+
     /*
     Current writer chain:
       FieldsConsumer
@@ -89,255 +71,48 @@ final class FreqProxTermsWriter extends 
                     -> IMPL: FormatPostingsPositionsWriter
     */
 
-    int start = 0;
-    while(start < numAllFields) {
-      final FieldInfo fieldInfo = allFields.get(start).fieldInfo;
-      final String fieldName = fieldInfo.name;
-
-      int end = start+1;
-      while(end < numAllFields && allFields.get(end).fieldInfo.name.equals(fieldName))
-        end++;
-      
-      FreqProxTermsWriterPerField[] fields = new FreqProxTermsWriterPerField[end-start];
-      for(int i=start;i<end;i++) {
-        fields[i-start] = allFields.get(i);
-
-        // Aggregate the storePayload as seen by the same
-        // field across multiple threads
-        fieldInfo.storePayloads |= fields[i-start].hasPayloads;
+    for (int fieldNumber = 0; fieldNumber < numAllFields; fieldNumber++) {
+      final FieldInfo fieldInfo = allFields.get(fieldNumber).fieldInfo;
+
+      final FreqProxTermsWriterPerField fieldWriter = allFields.get(fieldNumber);
+
+      // Aggregate the storePayload as seen by the same
+      // field across multiple threads
+      if (!fieldInfo.omitTermFreqAndPositions) {
+        fieldInfo.storePayloads |= fieldWriter.hasPayloads;
       }
 
       // If this field has postings then add them to the
       // segment
-      appendPostings(fieldName, state, fields, consumer);
-
-      for(int i=0;i<fields.length;i++) {
-        TermsHashPerField perField = fields[i].termsHashPerField;
-        int numPostings = perField.bytesHash.size();
-        perField.reset();
-        perField.shrinkHash(numPostings);
-        fields[i].reset();
-      }
+      fieldWriter.flush(fieldInfo.name, consumer, state);
 
-      start = end;
+      TermsHashPerField perField = fieldWriter.termsHashPerField;
+      assert termsHash == null || termsHash == perField.termsHash;
+      termsHash = perField.termsHash;
+      int numPostings = perField.bytesHash.size();
+      perField.reset();
+      perField.shrinkHash(numPostings);
+      fieldWriter.reset();
     }
 
-    for (Map.Entry<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> entry : threadsAndFields.entrySet()) {
-      FreqProxTermsWriterPerThread perThread = (FreqProxTermsWriterPerThread) entry.getKey();
-      perThread.termsHashPerThread.reset(true);
+    if (termsHash != null) {
+      termsHash.reset();
     }
     consumer.close();
   }
 
   BytesRef payload;
 
-  /* Walk through all unique text tokens (Posting
-   * instances) found in this field and serialize them
-   * into a single RAM segment. */
-  void appendPostings(String fieldName, SegmentWriteState state,
-                      FreqProxTermsWriterPerField[] fields,
-                      FieldsConsumer consumer)
-    throws CorruptIndexException, IOException {
-
-    int numFields = fields.length;
-
-    final BytesRef text = new BytesRef();
-
-    final FreqProxFieldMergeState[] mergeStates = new FreqProxFieldMergeState[numFields];
-
-    final TermsConsumer termsConsumer = consumer.addField(fields[0].fieldInfo);
-    final Comparator<BytesRef> termComp = termsConsumer.getComparator();
-
-    for(int i=0;i<numFields;i++) {
-      FreqProxFieldMergeState fms = mergeStates[i] = new FreqProxFieldMergeState(fields[i], termComp);
-
-      assert fms.field.fieldInfo == fields[0].fieldInfo;
-
-      // Should always be true
-      boolean result = fms.nextTerm();
-      assert result;
-    }
-
-    final Term protoTerm = new Term(fieldName);
-
-    FreqProxFieldMergeState[] termStates = new FreqProxFieldMergeState[numFields];
-
-    final boolean currentFieldOmitTermFreqAndPositions = fields[0].fieldInfo.omitTermFreqAndPositions;
-    //System.out.println("flush terms field=" + fields[0].fieldInfo.name);
-
-    final Map<Term,Integer> segDeletes;
-    if (state.segDeletes != null && state.segDeletes.terms.size() > 0) {
-      segDeletes = state.segDeletes.terms;
-    } else {
-      segDeletes = null;
-    }
-
-    // TODO: really TermsHashPerField should take over most
-    // of this loop, including merge sort of terms from
-    // multiple threads and interacting with the
-    // TermsConsumer, only calling out to us (passing us the
-    // DocsConsumer) to handle delivery of docs/positions
-    long sumTotalTermFreq = 0;
-    while(numFields > 0) {
-
-      // Get the next term to merge
-      termStates[0] = mergeStates[0];
-      int numToMerge = 1;
-
-      // TODO: pqueue
-      for(int i=1;i<numFields;i++) {
-        final int cmp = termComp.compare(mergeStates[i].text, termStates[0].text);
-        if (cmp < 0) {
-          termStates[0] = mergeStates[i];
-          numToMerge = 1;
-        } else if (cmp == 0) {
-          termStates[numToMerge++] = mergeStates[i];
-        }
-      }
-
-      // Need shallow copy here because termStates[0].text
-      // changes by the time we call finishTerm
-      text.bytes = termStates[0].text.bytes;
-      text.offset = termStates[0].text.offset;
-      text.length = termStates[0].text.length;  
-
-      //System.out.println("  term=" + text.toUnicodeString());
-      //System.out.println("  term=" + text.toString());
-
-      final PostingsConsumer postingsConsumer = termsConsumer.startTerm(text);
-
-      final int delDocLimit;
-      if (segDeletes != null) {
-        final Integer docIDUpto = segDeletes.get(protoTerm.createTerm(text));
-        if (docIDUpto != null) {
-          delDocLimit = docIDUpto;
-        } else {
-          delDocLimit = 0;
-        }
-      } else {
-        delDocLimit = 0;
-      }
-
-      // Now termStates has numToMerge FieldMergeStates
-      // which all share the same term.  Now we must
-      // interleave the docID streams.
-      int numDocs = 0;
-      long totTF = 0;
-      while(numToMerge > 0) {
-        
-        FreqProxFieldMergeState minState = termStates[0];
-        for(int i=1;i<numToMerge;i++) {
-          if (termStates[i].docID < minState.docID) {
-            minState = termStates[i];
-          }
-        }
-
-        final int termDocFreq = minState.termFreq;
-        numDocs++;
-
-        assert minState.docID < flushedDocCount: "doc=" + minState.docID + " maxDoc=" + flushedDocCount;
-
-        // NOTE: we could check here if the docID was
-        // deleted, and skip it.  However, this is somewhat
-        // dangerous because it can yield non-deterministic
-        // behavior since we may see the docID before we see
-        // the term that caused it to be deleted.  This
-        // would mean some (but not all) of its postings may
-        // make it into the index, which'd alter the docFreq
-        // for those terms.  We could fix this by doing two
-        // passes, ie first sweep marks all del docs, and
-        // 2nd sweep does the real flush, but I suspect
-        // that'd add too much time to flush.
-
-        postingsConsumer.startDoc(minState.docID, termDocFreq);
-        if (minState.docID < delDocLimit) {
-          // Mark it deleted.  TODO: we could also skip
-          // writing its postings; this would be
-          // deterministic (just for this Term's docs).
-          if (state.deletedDocs == null) {
-            state.deletedDocs = new BitVector(state.numDocs);
-          }
-          state.deletedDocs.set(minState.docID);
-        }
-
-        final ByteSliceReader prox = minState.prox;
-
-        // Carefully copy over the prox + payload info,
-        // changing the format to match Lucene's segment
-        // format.
-        if (!currentFieldOmitTermFreqAndPositions) {
-          // omitTermFreqAndPositions == false so we do write positions &
-          // payload          
-          int position = 0;
-          totTF += termDocFreq;
-          for(int j=0;j<termDocFreq;j++) {
-            final int code = prox.readVInt();
-            position += code >> 1;
-            //System.out.println("    pos=" + position);
-
-            final int payloadLength;
-            final BytesRef thisPayload;
-
-            if ((code & 1) != 0) {
-              // This position has a payload
-              payloadLength = prox.readVInt();  
-              
-              if (payload == null) {
-                payload = new BytesRef();
-                payload.bytes = new byte[payloadLength];
-              } else if (payload.bytes.length < payloadLength) {
-                payload.grow(payloadLength);
-              }
-
-              prox.readBytes(payload.bytes, 0, payloadLength);
-              payload.length = payloadLength;
-              thisPayload = payload;
-
-            } else {
-              payloadLength = 0;
-              thisPayload = null;
-            }
-
-            postingsConsumer.addPosition(position, thisPayload);
-          } //End for
-
-          postingsConsumer.finishDoc();
-        }
-
-        if (!minState.nextDoc()) {
-
-          // Remove from termStates
-          int upto = 0;
-          // TODO: inefficient O(N) where N = number of
-          // threads that had seen this term:
-          for(int i=0;i<numToMerge;i++) {
-            if (termStates[i] != minState) {
-              termStates[upto++] = termStates[i];
-            }
-          }
-          numToMerge--;
-          assert upto == numToMerge;
-
-          // Advance this state to the next term
-
-          if (!minState.nextTerm()) {
-            // OK, no more terms, so remove from mergeStates
-            // as well
-            upto = 0;
-            for(int i=0;i<numFields;i++)
-              if (mergeStates[i] != minState)
-                mergeStates[upto++] = mergeStates[i];
-            numFields--;
-            assert upto == numFields;
-          }
-        }
-      }
+  @Override
+  public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo) {
+    return new FreqProxTermsWriterPerField(termsHashPerField, this, fieldInfo);
+  }
 
-      assert numDocs > 0;
-      termsConsumer.finishTerm(text, new TermStats(numDocs, totTF));
-      sumTotalTermFreq += totTF;
-    }
+  @Override
+  void finishDocument(TermsHash termsHash) throws IOException {
+  }
 
-    termsConsumer.finish(sumTotalTermFreq);
+  @Override
+  void startDocument() throws IOException {
   }
 }

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java?rev=1103112&r1=1103111&r2=1103112&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java Sat May 14 13:51:35 2011
@@ -18,9 +18,17 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
+import java.util.Comparator;
+import java.util.Map;
 
 import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
 import org.apache.lucene.document.Fieldable;
+import org.apache.lucene.index.codecs.FieldsConsumer;
+import org.apache.lucene.index.codecs.PostingsConsumer;
+import org.apache.lucene.index.codecs.TermStats;
+import org.apache.lucene.index.codecs.TermsConsumer;
+import org.apache.lucene.util.BitVector;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.RamUsageEstimator;
 
 // TODO: break into separate freq and prox writers as
@@ -28,17 +36,17 @@ import org.apache.lucene.util.RamUsageEs
 // be configured as any number of files 1..N
 final class FreqProxTermsWriterPerField extends TermsHashConsumerPerField implements Comparable<FreqProxTermsWriterPerField> {
 
-  final FreqProxTermsWriterPerThread perThread;
+  final FreqProxTermsWriter parent;
   final TermsHashPerField termsHashPerField;
   final FieldInfo fieldInfo;
-  final DocumentsWriter.DocState docState;
+  final DocumentsWriterPerThread.DocState docState;
   final FieldInvertState fieldState;
   boolean omitTermFreqAndPositions;
   PayloadAttribute payloadAttribute;
 
-  public FreqProxTermsWriterPerField(TermsHashPerField termsHashPerField, FreqProxTermsWriterPerThread perThread, FieldInfo fieldInfo) {
+  public FreqProxTermsWriterPerField(TermsHashPerField termsHashPerField, FreqProxTermsWriter parent, FieldInfo fieldInfo) {
     this.termsHashPerField = termsHashPerField;
-    this.perThread = perThread;
+    this.parent = parent;
     this.fieldInfo = fieldInfo;
     docState = termsHashPerField.docState;
     fieldState = termsHashPerField.fieldState;
@@ -78,8 +86,8 @@ final class FreqProxTermsWriterPerField 
       if (fields[i].isIndexed())
         return true;
     return false;
-  }     
-  
+  }
+
   @Override
   void start(Fieldable f) {
     if (fieldState.attributeSource.hasAttribute(PayloadAttribute.class)) {
@@ -96,18 +104,18 @@ final class FreqProxTermsWriterPerField 
     } else {
       payload = payloadAttribute.getPayload();
     }
-    
+
     if (payload != null && payload.length > 0) {
       termsHashPerField.writeVInt(1, (proxCode<<1)|1);
       termsHashPerField.writeVInt(1, payload.length);
       termsHashPerField.writeBytes(1, payload.data, payload.offset, payload.length);
-      hasPayloads = true;      
+      hasPayloads = true;
     } else
       termsHashPerField.writeVInt(1, proxCode<<1);
-    
+
     FreqProxPostingsArray postings = (FreqProxPostingsArray) termsHashPerField.postingsArray;
     postings.lastPositions[termID] = fieldState.position;
-    
+
   }
 
   @Override
@@ -115,7 +123,7 @@ final class FreqProxTermsWriterPerField 
     // First time we're seeing this term since the last
     // flush
     assert docState.testPoint("FreqProxTermsWriterPerField.newTerm start");
-    
+
     FreqProxPostingsArray postings = (FreqProxPostingsArray) termsHashPerField.postingsArray;
     postings.lastDocIDs[termID] = docState.docID;
     if (omitTermFreqAndPositions) {
@@ -132,9 +140,9 @@ final class FreqProxTermsWriterPerField 
   void addTerm(final int termID) {
 
     assert docState.testPoint("FreqProxTermsWriterPerField.addTerm start");
-    
+
     FreqProxPostingsArray postings = (FreqProxPostingsArray) termsHashPerField.postingsArray;
-    
+
     assert omitTermFreqAndPositions || postings.docFreqs[termID] > 0;
 
     if (omitTermFreqAndPositions) {
@@ -169,7 +177,7 @@ final class FreqProxTermsWriterPerField 
       }
     }
   }
-  
+
   @Override
   ParallelPostingsArray createPostingsArray(int size) {
     return new FreqProxPostingsArray(size);
@@ -212,7 +220,180 @@ final class FreqProxTermsWriterPerField 
       return ParallelPostingsArray.BYTES_PER_POSTING + 4 * RamUsageEstimator.NUM_BYTES_INT;
     }
   }
-  
+
   public void abort() {}
+
+  BytesRef payload;
+
+  /* Walk through all unique text tokens (Posting
+   * instances) found in this field and serialize them
+   * into a single RAM segment. */
+  void flush(String fieldName, FieldsConsumer consumer,  final SegmentWriteState state)
+    throws CorruptIndexException, IOException {
+
+    final TermsConsumer termsConsumer = consumer.addField(fieldInfo);
+    final Comparator<BytesRef> termComp = termsConsumer.getComparator();
+
+    final Term protoTerm = new Term(fieldName);
+
+    final boolean currentFieldOmitTermFreqAndPositions = fieldInfo.omitTermFreqAndPositions;
+
+    final Map<Term,Integer> segDeletes;
+    if (state.segDeletes != null && state.segDeletes.terms.size() > 0) {
+      segDeletes = state.segDeletes.terms;
+    } else {
+      segDeletes = null;
+    }
+
+    final int[] termIDs = termsHashPerField.sortPostings(termComp);
+    final int numTerms = termsHashPerField.bytesHash.size();
+    final BytesRef text = new BytesRef();
+    final FreqProxPostingsArray postings = (FreqProxPostingsArray) termsHashPerField.postingsArray;
+    final ByteSliceReader freq = new ByteSliceReader();
+    final ByteSliceReader prox = new ByteSliceReader();
+
+    long sumTotalTermFreq = 0;
+    for (int i = 0; i < numTerms; i++) {
+      final int termID = termIDs[i];
+      // Get BytesRef
+      final int textStart = postings.textStarts[termID];
+      termsHashPerField.bytePool.setBytesRef(text, textStart);
+
+      termsHashPerField.initReader(freq, termID, 0);
+      if (!fieldInfo.omitTermFreqAndPositions) {
+        termsHashPerField.initReader(prox, termID, 1);
+      }
+
+      // TODO: really TermsHashPerField should take over most
+      // of this loop, including merge sort of terms from
+      // multiple threads and interacting with the
+      // TermsConsumer, only calling out to us (passing us the
+      // DocsConsumer) to handle delivery of docs/positions
+
+      final PostingsConsumer postingsConsumer = termsConsumer.startTerm(text);
+
+      final int delDocLimit;
+      if (segDeletes != null) {
+        final Integer docIDUpto = segDeletes.get(protoTerm.createTerm(text));
+        if (docIDUpto != null) {
+          delDocLimit = docIDUpto;
+        } else {
+          delDocLimit = 0;
+        }
+      } else {
+        delDocLimit = 0;
+      }
+
+      // Now termStates has numToMerge FieldMergeStates
+      // which all share the same term.  Now we must
+      // interleave the docID streams.
+      int numDocs = 0;
+      long totTF = 0;
+      int docID = 0;
+      int termFreq = 0;
+
+      while(true) {
+        if (freq.eof()) {
+          if (postings.lastDocCodes[termID] != -1) {
+            // Return last doc
+            docID = postings.lastDocIDs[termID];
+            if (!omitTermFreqAndPositions) {
+              termFreq = postings.docFreqs[termID];
+            }
+            postings.lastDocCodes[termID] = -1;
+          } else {
+            // EOF
+            break;
+          }
+        } else {
+          final int code = freq.readVInt();
+          if (omitTermFreqAndPositions) {
+            docID += code;
+          } else {
+            docID += code >>> 1;
+            if ((code & 1) != 0) {
+              termFreq = 1;
+            } else {
+              termFreq = freq.readVInt();
+            }
+          }
+
+          assert docID != postings.lastDocIDs[termID];
+        }
+
+        numDocs++;
+        assert docID < state.numDocs: "doc=" + docID + " maxDoc=" + state.numDocs;
+        final int termDocFreq = termFreq;
+
+        // NOTE: we could check here if the docID was
+        // deleted, and skip it.  However, this is somewhat
+        // dangerous because it can yield non-deterministic
+        // behavior since we may see the docID before we see
+        // the term that caused it to be deleted.  This
+        // would mean some (but not all) of its postings may
+        // make it into the index, which'd alter the docFreq
+        // for those terms.  We could fix this by doing two
+        // passes, ie first sweep marks all del docs, and
+        // 2nd sweep does the real flush, but I suspect
+        // that'd add too much time to flush.
+        postingsConsumer.startDoc(docID, termDocFreq);
+        if (docID < delDocLimit) {
+          // Mark it deleted.  TODO: we could also skip
+          // writing its postings; this would be
+          // deterministic (just for this Term's docs).
+          if (state.deletedDocs == null) {
+            state.deletedDocs = new BitVector(state.numDocs);
+          }
+          state.deletedDocs.set(docID);
+        }
+
+        // Carefully copy over the prox + payload info,
+        // changing the format to match Lucene's segment
+        // format.
+        if (!currentFieldOmitTermFreqAndPositions) {
+          // omitTermFreqAndPositions == false so we do write positions &
+          // payload
+          int position = 0;
+          totTF += termDocFreq;
+          for(int j=0;j<termDocFreq;j++) {
+            final int code = prox.readVInt();
+            position += code >> 1;
+
+            final int payloadLength;
+            final BytesRef thisPayload;
+
+            if ((code & 1) != 0) {
+              // This position has a payload
+              payloadLength = prox.readVInt();
+
+              if (payload == null) {
+                payload = new BytesRef();
+                payload.bytes = new byte[payloadLength];
+              } else if (payload.bytes.length < payloadLength) {
+                payload.grow(payloadLength);
+              }
+
+              prox.readBytes(payload.bytes, 0, payloadLength);
+              payload.length = payloadLength;
+              thisPayload = payload;
+
+            } else {
+              payloadLength = 0;
+              thisPayload = null;
+            }
+
+            postingsConsumer.addPosition(position, thisPayload);
+          }
+
+          postingsConsumer.finishDoc();
+        }
+      }
+      termsConsumer.finishTerm(text, new TermStats(numDocs, totTF));
+      sumTotalTermFreq += totTF;
+    }
+
+    termsConsumer.finish(sumTotalTermFreq);
+  }
+
 }
 

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=1103112&r1=1103111&r2=1103112&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java Sat May 14 13:51:35 2011
@@ -21,7 +21,14 @@ import java.io.FileNotFoundException;
 import java.io.FilenameFilter;
 import java.io.IOException;
 import java.io.PrintStream;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.store.Directory;
@@ -49,12 +56,12 @@ import org.apache.lucene.util.Collection
  * (IndexDeletionPolicy) is consulted on creation (onInit)
  * and once per commit (onCommit), to decide when a commit
  * should be removed.
- * 
+ *
  * It is the business of the IndexDeletionPolicy to choose
  * when to delete commit points.  The actual mechanics of
  * file deletion, retrying, etc, derived from the deletion
  * of commit points is the business of the IndexFileDeleter.
- * 
+ *
  * The current default deletion policy is {@link
  * KeepOnlyLastCommitDeletionPolicy}, which removes all
  * prior commits when a new commit has completed.  This
@@ -72,7 +79,7 @@ final class IndexFileDeleter {
    * so we will retry them again later: */
   private List<String> deletable;
 
-  /* Reference count for all files in the index.  
+  /* Reference count for all files in the index.
    * Counts how many existing commits reference a file.
    **/
   private Map<String, RefCount> refCounts = new HashMap<String, RefCount>();
@@ -88,7 +95,7 @@ final class IndexFileDeleter {
    * non-commit checkpoint: */
   private List<Collection<String>> lastFiles = new ArrayList<Collection<String>>();
 
-  /* Commits that the IndexDeletionPolicy have decided to delete: */ 
+  /* Commits that the IndexDeletionPolicy have decided to delete: */
   private List<CommitPoint> commitsToDelete = new ArrayList<CommitPoint>();
 
   private PrintStream infoStream;
@@ -108,7 +115,7 @@ final class IndexFileDeleter {
       message("setInfoStream deletionPolicy=" + policy);
     }
   }
-  
+
   private void message(String message) {
     infoStream.println("IFD [" + new Date() + "; " + Thread.currentThread().getName() + "]: " + message);
   }
@@ -139,12 +146,12 @@ final class IndexFileDeleter {
     // counts:
     long currentGen = segmentInfos.getGeneration();
     indexFilenameFilter = new IndexFileNameFilter(codecs);
-    
+
     CommitPoint currentCommitPoint = null;
     String[] files = null;
     try {
       files = directory.listAll();
-    } catch (NoSuchDirectoryException e) {  
+    } catch (NoSuchDirectoryException e) {
       // it means the directory is empty, so ignore it.
       files = new String[0];
     }
@@ -152,7 +159,7 @@ final class IndexFileDeleter {
     for (String fileName : files) {
 
       if ((indexFilenameFilter.accept(null, fileName)) && !fileName.endsWith("write.lock") && !fileName.equals(IndexFileNames.SEGMENTS_GEN)) {
-        
+
         // Add this file to refCounts with initial count 0:
         getRefCount(fileName);
 
@@ -190,7 +197,31 @@ final class IndexFileDeleter {
             }
           }
           if (sis != null) {
-            CommitPoint commitPoint = new CommitPoint(commitsToDelete, directory, sis);
+            final SegmentInfos infos = sis;
+            for (SegmentInfo segmentInfo : infos) {
+              try {
+                /*
+                 * Force FI to load for each segment since we could see a
+                 * segments file and load successfully above if the files are
+                 * still referenced when they are deleted and the os doesn't let
+                 * you delete them. Yet its likely that fnm files are removed
+                 * while seg file is still around Since LUCENE-2984 we need FI
+                 * to find out if a seg has vectors and prox so we need those
+                 * files to be opened for a commit point.
+                 */
+                segmentInfo.getFieldInfos();
+              } catch (FileNotFoundException e) {
+                refresh(segmentInfo.name);
+                sis = null;
+                if (infoStream != null) {
+                  message("init: hit FileNotFoundException when loading commit \"" + fileName + "\"; skipping this commit point");
+                }
+              }
+            }
+           
+          }
+          if (sis != null) {
+            final CommitPoint commitPoint = new CommitPoint(commitsToDelete, directory, sis);
             if (sis.getGeneration() == segmentInfos.getGeneration()) {
               currentCommitPoint = commitPoint;
             }
@@ -233,7 +264,7 @@ final class IndexFileDeleter {
     // Now delete anything with ref count at 0.  These are
     // presumably abandoned files eg due to crash of
     // IndexWriter.
-    for(Map.Entry<String, RefCount> entry : refCounts.entrySet() ) {  
+    for(Map.Entry<String, RefCount> entry : refCounts.entrySet() ) {
       RefCount rc = entry.getValue();
       final String fileName = entry.getKey();
       if (0 == rc.count) {
@@ -253,7 +284,7 @@ final class IndexFileDeleter {
     // Always protect the incoming segmentInfos since
     // sometime it may not be the most recent commit
     checkpoint(segmentInfos, false);
-    
+
     startingCommitDeleted = currentCommitPoint == null ? false : currentCommitPoint.isDeleted();
 
     deleteCommits();
@@ -327,7 +358,7 @@ final class IndexFileDeleter {
       segmentPrefix1 = null;
       segmentPrefix2 = null;
     }
-    
+
     for(int i=0;i<files.length;i++) {
       String fileName = files[i];
       if ((segmentName == null || fileName.startsWith(segmentPrefix1) || fileName.startsWith(segmentPrefix2)) &&
@@ -379,7 +410,7 @@ final class IndexFileDeleter {
       deleteCommits();
     }
   }
-  
+
   public void deletePendingFiles() throws IOException {
     if (deletable != null) {
       List<String> oldDeletable = deletable;
@@ -397,7 +428,7 @@ final class IndexFileDeleter {
   /**
    * For definition of "check point" see IndexWriter comments:
    * "Clarification: Check Points (and commits)".
-   * 
+   *
    * Writer calls this when it has made a "consistent
    * change" to the index, meaning new files are written to
    * the index and the in-memory SegmentInfos have been
@@ -417,7 +448,7 @@ final class IndexFileDeleter {
   public void checkpoint(SegmentInfos segmentInfos, boolean isCommit) throws IOException {
 
     if (infoStream != null) {
-      message("now checkpoint \"" + segmentInfos.getCurrentSegmentFileName() + "\" [" + segmentInfos.size() + " segments " + "; isCommit = " + isCommit + "]");
+      message("now checkpoint \"" + segmentInfos + "\" [" + segmentInfos.size() + " segments " + "; isCommit = " + isCommit + "]");
     }
 
     // Try again now to delete any previously un-deletable

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/IndexReader.java?rev=1103112&r1=1103111&r2=1103112&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/IndexReader.java Sat May 14 13:51:35 2011
@@ -919,6 +919,22 @@ public abstract class IndexReader implem
     }
   }
 
+  /**
+   * Returns <code>true</code> if an index exists at the specified directory.
+   * @param  directory the directory to check for an index
+   * @param  codecProvider provides a CodecProvider in case the index uses non-core codecs
+   * @return <code>true</code> if an index exists; <code>false</code> otherwise
+   * @throws IOException if there is a problem with accessing the index
+   */
+  public static boolean indexExists(Directory directory, CodecProvider codecProvider) throws IOException {
+    try {
+      new SegmentInfos().read(directory, codecProvider);
+      return true;
+    } catch (IOException ioe) {
+      return false;
+    }
+  }
+
   /** Returns the number of documents in this index. */
   public abstract int numDocs();
 
@@ -1411,7 +1427,7 @@ public abstract class IndexReader implem
       cfr = new CompoundFileReader(dir, filename);
 
       String [] files = cfr.listAll();
-      ArrayUtil.quickSort(files);   // sort the array of filename so that the output is more readable
+      ArrayUtil.mergeSort(files);   // sort the array of filename so that the output is more readable
 
       for (int i = 0; i < files.length; ++i) {
         long len = cfr.fileLength(files[i]);