You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ja...@apache.org on 2013/05/30 09:53:46 UTC

svn commit: r1487777 [16/50] - in /lucene/dev/branches/security: ./ dev-tools/ dev-tools/eclipse/dot.settings/ dev-tools/idea/.idea/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/replicator/ dev-tools/maven/ dev-tools/maven/lucene/ dev-tools/ma...

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java Thu May 30 07:53:18 2013
@@ -17,7 +17,13 @@ package org.apache.lucene.codecs.lucene4
  * limitations under the License.
  */
 
+import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesConsumer.DELTA_COMPRESSED;
+import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesConsumer.GCD_COMPRESSED;
+import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesConsumer.TABLE_COMPRESSED;
+import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesConsumer.UNCOMPRESSED;
+
 import java.io.IOException;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -25,6 +31,8 @@ import org.apache.lucene.codecs.CodecUti
 import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
@@ -32,8 +40,10 @@ import org.apache.lucene.index.NumericDo
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.IntsRef;
@@ -75,14 +85,16 @@ class Lucene42DocValuesProducer extends 
     // read in the entries from the metadata file.
     IndexInput in = state.directory.openInput(metaName, state.context);
     boolean success = false;
+    final int version;
     try {
-      CodecUtil.checkHeader(in, metaCodec, 
-                                Lucene42DocValuesConsumer.VERSION_START,
-                                Lucene42DocValuesConsumer.VERSION_START);
+      version = CodecUtil.checkHeader(in, metaCodec, 
+                                      Lucene42DocValuesConsumer.VERSION_START,
+                                      Lucene42DocValuesConsumer.VERSION_CURRENT);
       numerics = new HashMap<Integer,NumericEntry>();
       binaries = new HashMap<Integer,BinaryEntry>();
       fsts = new HashMap<Integer,FSTEntry>();
       readFields(in, state.fieldInfos);
+
       success = true;
     } finally {
       if (success) {
@@ -91,12 +103,24 @@ class Lucene42DocValuesProducer extends 
         IOUtils.closeWhileHandlingException(in);
       }
     }
-    
-    String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
-    data = state.directory.openInput(dataName, state.context);
-    CodecUtil.checkHeader(data, dataCodec, 
-                                Lucene42DocValuesConsumer.VERSION_START,
-                                Lucene42DocValuesConsumer.VERSION_START);
+
+    success = false;
+    try {
+      String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
+      data = state.directory.openInput(dataName, state.context);
+      final int version2 = CodecUtil.checkHeader(data, dataCodec, 
+                                                 Lucene42DocValuesConsumer.VERSION_START,
+                                                 Lucene42DocValuesConsumer.VERSION_CURRENT);
+      if (version != version2) {
+        throw new CorruptIndexException("Format versions mismatch");
+      }
+
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(this.data);
+      }
+    }
   }
   
   private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
@@ -107,6 +131,15 @@ class Lucene42DocValuesProducer extends 
         NumericEntry entry = new NumericEntry();
         entry.offset = meta.readLong();
         entry.format = meta.readByte();
+        switch(entry.format) {
+          case DELTA_COMPRESSED:
+          case TABLE_COMPRESSED:
+          case GCD_COMPRESSED:
+          case UNCOMPRESSED:
+               break;
+          default:
+               throw new CorruptIndexException("Unknown format: " + entry.format + ", input=" + meta);
+        }
         if (entry.format != Lucene42DocValuesConsumer.UNCOMPRESSED) {
           entry.packedIntsVersion = meta.readVInt();
         }
@@ -147,41 +180,56 @@ class Lucene42DocValuesProducer extends 
   private NumericDocValues loadNumeric(FieldInfo field) throws IOException {
     NumericEntry entry = numerics.get(field.number);
     data.seek(entry.offset);
-    if (entry.format == Lucene42DocValuesConsumer.TABLE_COMPRESSED) {
-      int size = data.readVInt();
-      final long decode[] = new long[size];
-      for (int i = 0; i < decode.length; i++) {
-        decode[i] = data.readLong();
-      }
-      final int formatID = data.readVInt();
-      final int bitsPerValue = data.readVInt();
-      final PackedInts.Reader reader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), entry.packedIntsVersion, maxDoc, bitsPerValue);
-      return new NumericDocValues() {
-        @Override
-        public long get(int docID) {
-          return decode[(int)reader.get(docID)];
-        }
-      };
-    } else if (entry.format == Lucene42DocValuesConsumer.DELTA_COMPRESSED) {
-      final int blockSize = data.readVInt();
-      final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, blockSize, maxDoc, false);
-      return new NumericDocValues() {
-        @Override
-        public long get(int docID) {
-          return reader.get(docID);
-        }
-      };
-    } else if (entry.format == Lucene42DocValuesConsumer.UNCOMPRESSED) {
-      final byte bytes[] = new byte[maxDoc];
-      data.readBytes(bytes, 0, bytes.length);
-      return new NumericDocValues() {
-        @Override
-        public long get(int docID) {
-          return bytes[docID];
-        }
-      };
-    } else {
-      throw new IllegalStateException();
+    switch (entry.format) {
+      case TABLE_COMPRESSED:
+        int size = data.readVInt();
+        if (size > 256) {
+          throw new CorruptIndexException("TABLE_COMPRESSED cannot have more than 256 distinct values, input=" + data);
+        }
+        final long decode[] = new long[size];
+        for (int i = 0; i < decode.length; i++) {
+          decode[i] = data.readLong();
+        }
+        final int formatID = data.readVInt();
+        final int bitsPerValue = data.readVInt();
+        final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), entry.packedIntsVersion, maxDoc, bitsPerValue);
+        return new NumericDocValues() {
+          @Override
+          public long get(int docID) {
+            return decode[(int)ordsReader.get(docID)];
+          }
+        };
+      case DELTA_COMPRESSED:
+        final int blockSize = data.readVInt();
+        final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, blockSize, maxDoc, false);
+        return new NumericDocValues() {
+          @Override
+          public long get(int docID) {
+            return reader.get(docID);
+          }
+        };
+      case UNCOMPRESSED:
+        final byte bytes[] = new byte[maxDoc];
+        data.readBytes(bytes, 0, bytes.length);
+        return new NumericDocValues() {
+          @Override
+          public long get(int docID) {
+            return bytes[docID];
+          }
+        };
+      case GCD_COMPRESSED:
+        final long min = data.readLong();
+        final long mult = data.readLong();
+        final int quotientBlockSize = data.readVInt();
+        final BlockPackedReader quotientReader = new BlockPackedReader(data, entry.packedIntsVersion, quotientBlockSize, maxDoc, false);
+        return new NumericDocValues() {
+          @Override
+          public long get(int docID) {
+            return min + mult * quotientReader.get(docID);
+          }
+        };
+      default:
+        throw new AssertionError();
     }
   }
 
@@ -285,6 +333,11 @@ class Lucene42DocValuesProducer extends 
       public int getValueCount() {
         return (int)entry.numOrds;
       }
+
+      @Override
+      public TermsEnum termsEnum() {
+        return new FSTTermsEnum(fst);
+      }
     };
   }
   
@@ -369,6 +422,11 @@ class Lucene42DocValuesProducer extends 
       public long getValueCount() {
         return entry.numOrds;
       }
+
+      @Override
+      public TermsEnum termsEnum() {
+        return new FSTTermsEnum(fst);
+      }
     };
   }
 
@@ -396,4 +454,106 @@ class Lucene42DocValuesProducer extends 
     long offset;
     long numOrds;
   }
+  
+  // exposes FSTEnum directly as a TermsEnum: avoids binary-search next()
+  static class FSTTermsEnum extends TermsEnum {
+    final BytesRefFSTEnum<Long> in;
+    
+    // this is all for the complicated seek(ord)...
+    // maybe we should add a FSTEnum that supports this operation?
+    final FST<Long> fst;
+    final FST.BytesReader bytesReader;
+    final Arc<Long> firstArc = new Arc<Long>();
+    final Arc<Long> scratchArc = new Arc<Long>();
+    final IntsRef scratchInts = new IntsRef();
+    final BytesRef scratchBytes = new BytesRef();
+    
+    FSTTermsEnum(FST<Long> fst) {
+      this.fst = fst;
+      in = new BytesRefFSTEnum<Long>(fst);
+      bytesReader = fst.getBytesReader();
+    }
+
+    @Override
+    public BytesRef next() throws IOException {
+      InputOutput<Long> io = in.next();
+      if (io == null) {
+        return null;
+      } else {
+        return io.input;
+      }
+    }
+
+    @Override
+    public Comparator<BytesRef> getComparator() {
+      return BytesRef.getUTF8SortedAsUnicodeComparator();
+    }
+
+    @Override
+    public SeekStatus seekCeil(BytesRef text, boolean useCache) throws IOException {
+      if (in.seekCeil(text) == null) {
+        return SeekStatus.END;
+      } else if (term().equals(text)) {
+        // TODO: add SeekStatus to FSTEnum like in https://issues.apache.org/jira/browse/LUCENE-3729
+        // to remove this comparision?
+        return SeekStatus.FOUND;
+      } else {
+        return SeekStatus.NOT_FOUND;
+      }
+    }
+
+    @Override
+    public boolean seekExact(BytesRef text, boolean useCache) throws IOException {
+      if (in.seekExact(text) == null) {
+        return false;
+      } else {
+        return true;
+      }
+    }
+
+    @Override
+    public void seekExact(long ord) throws IOException {
+      // TODO: would be better to make this simpler and faster.
+      // but we dont want to introduce a bug that corrupts our enum state!
+      bytesReader.setPosition(0);
+      fst.getFirstArc(firstArc);
+      IntsRef output = Util.getByOutput(fst, ord, bytesReader, firstArc, scratchArc, scratchInts);
+      scratchBytes.bytes = new byte[output.length];
+      scratchBytes.offset = 0;
+      scratchBytes.length = 0;
+      Util.toBytesRef(output, scratchBytes);
+      // TODO: we could do this lazily, better to try to push into FSTEnum though?
+      in.seekExact(scratchBytes);
+    }
+
+    @Override
+    public BytesRef term() throws IOException {
+      return in.current().input;
+    }
+
+    @Override
+    public long ord() throws IOException {
+      return in.current().output;
+    }
+
+    @Override
+    public int docFreq() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long totalTermFreq() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+  }
 }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java Thu May 30 07:53:18 2013
@@ -107,6 +107,33 @@ public abstract class AtomicReader exten
       return 0;
     }
   }
+  
+  @Override
+  public final long getSumDocFreq(String field) throws IOException {
+    final Terms terms = terms(field);
+    if (terms == null) {
+      return 0;
+    }
+    return terms.getSumDocFreq();
+  }
+  
+  @Override
+  public final int getDocCount(String field) throws IOException {
+    final Terms terms = terms(field);
+    if (terms == null) {
+      return 0;
+    }
+    return terms.getDocCount();
+  }
+  
+  @Override
+  public final long getSumTotalTermFreq(String field) throws IOException {
+    final Terms terms = terms(field);
+    if (terms == null) {
+      return 0;
+    }
+    return terms.getSumTotalTermFreq();
+  }
 
   /** This may return null if the field does not exist.*/
   public final Terms terms(String field) throws IOException {

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java Thu May 30 07:53:18 2013
@@ -51,7 +51,6 @@ public abstract class BaseCompositeReade
   private final int[] starts;       // 1st docno for each reader
   private final int maxDoc;
   private final int numDocs;
-  private final boolean hasDeletions;
 
   /** List view solely for {@link #getSequentialSubReaders()},
    * for effectiveness the array is used internally. */
@@ -70,7 +69,6 @@ public abstract class BaseCompositeReade
     this.subReadersList = Collections.unmodifiableList(Arrays.asList(subReaders));
     starts = new int[subReaders.length + 1];    // build starts array
     int maxDoc = 0, numDocs = 0;
-    boolean hasDeletions = false;
     for (int i = 0; i < subReaders.length; i++) {
       starts[i] = maxDoc;
       final IndexReader r = subReaders[i];
@@ -79,15 +77,11 @@ public abstract class BaseCompositeReade
         throw new IllegalArgumentException("Too many documents, composite IndexReaders cannot exceed " + Integer.MAX_VALUE);
       }
       numDocs += r.numDocs();    // compute numDocs
-      if (r.hasDeletions()) {
-        hasDeletions = true;
-      }
       r.registerParentReader(this);
     }
     starts[subReaders.length] = maxDoc;
     this.maxDoc = maxDoc;
     this.numDocs = numDocs;
-    this.hasDeletions = hasDeletions;
   }
 
   @Override
@@ -117,12 +111,6 @@ public abstract class BaseCompositeReade
   }
 
   @Override
-  public final boolean hasDeletions() {
-    // Don't call ensureOpen() here (it could affect performance)
-    return hasDeletions;
-  }
-
-  @Override
   public final int docFreq(Term term) throws IOException {
     ensureOpen();
     int total = 0;          // sum freqs in subreaders
@@ -145,7 +133,49 @@ public abstract class BaseCompositeReade
     }
     return total;
   }
+  
+  @Override
+  public final long getSumDocFreq(String field) throws IOException {
+    ensureOpen();
+    long total = 0; // sum doc freqs in subreaders
+    for (R reader : subReaders) {
+      long sub = reader.getSumDocFreq(field);
+      if (sub == -1) {
+        return -1; // if any of the subs doesn't support it, return -1
+      }
+      total += sub;
+    }
+    return total;
+  }
+  
+  @Override
+  public final int getDocCount(String field) throws IOException {
+    ensureOpen();
+    int total = 0; // sum doc counts in subreaders
+    for (R reader : subReaders) {
+      int sub = reader.getDocCount(field);
+      if (sub == -1) {
+        return -1; // if any of the subs doesn't support it, return -1
+      }
+      total += sub;
+    }
+    return total;
+  }
 
+  @Override
+  public final long getSumTotalTermFreq(String field) throws IOException {
+    ensureOpen();
+    long total = 0; // sum doc total term freqs in subreaders
+    for (R reader : subReaders) {
+      long sub = reader.getSumTotalTermFreq(field);
+      if (sub == -1) {
+        return -1; // if any of the subs doesn't support it, return -1
+      }
+      total += sub;
+    }
+    return total;
+  }
+  
   /** Helper method for subclasses to get the corresponding reader for a doc ID */
   protected final int readerIndex(int docID) {
     if (docID < 0 || docID >= maxDoc) {

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java Thu May 30 07:53:18 2013
@@ -134,14 +134,7 @@ class BufferedDeletesStream {
   private static final Comparator<SegmentInfoPerCommit> sortSegInfoByDelGen = new Comparator<SegmentInfoPerCommit>() {
     @Override
     public int compare(SegmentInfoPerCommit si1, SegmentInfoPerCommit si2) {
-      final long cmp = si1.getBufferedDeletesGen() - si2.getBufferedDeletesGen();
-      if (cmp > 0) {
-        return 1;
-      } else if (cmp < 0) {
-        return -1;
-      } else {
-        return 0;
-      }
+      return Long.compare(si1.getBufferedDeletesGen(), si2.getBufferedDeletesGen());
     }
   };
   

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/CompositeReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/CompositeReader.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/CompositeReader.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/CompositeReader.java Thu May 30 07:53:18 2013
@@ -66,7 +66,13 @@ public abstract class CompositeReader ex
   @Override
   public String toString() {
     final StringBuilder buffer = new StringBuilder();
-    buffer.append(getClass().getSimpleName());
+    // walk up through class hierarchy to get a non-empty simple name (anonymous classes have no name):
+    for (Class<?> clazz = getClass(); clazz != null; clazz = clazz.getSuperclass()) {
+      if (!clazz.isAnonymousClass()) {
+        buffer.append(clazz.getSimpleName());
+        break;
+      }
+    }
     buffer.append('(');
     final List<? extends IndexReader> subReaders = getSequentialSubReaders();
     assert subReaders != null;

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java Thu May 30 07:53:18 2013
@@ -184,7 +184,7 @@ public class ConcurrentMergeScheduler ex
     }
 
     // Sort the merge threads in descending order.
-    CollectionUtil.mergeSort(activeMerges, compareByMergeDocCount);
+    CollectionUtil.timSort(activeMerges, compareByMergeDocCount);
     
     int pri = mergeThreadPriority;
     final int activeMergeCount = activeMerges.size();
@@ -561,4 +561,13 @@ public class ConcurrentMergeScheduler ex
     sb.append("mergeThreadPriority=").append(mergeThreadPriority);
     return sb.toString();
   }
+
+  @Override
+  public MergeScheduler clone() {
+    ConcurrentMergeScheduler clone = (ConcurrentMergeScheduler) super.clone();
+    clone.writer = null;
+    clone.dir = null;
+    clone.mergeThreads = new ArrayList<MergeThread>();
+    return clone;
+  }
 }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/CorruptIndexException.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/CorruptIndexException.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/CorruptIndexException.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/CorruptIndexException.java Thu May 30 07:53:18 2013
@@ -1,3 +1,5 @@
+package org.apache.lucene.index;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -15,8 +17,6 @@
  * limitations under the License.
  */
 
-package org.apache.lucene.index;
-
 import java.io.IOException;
 
 /**

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java Thu May 30 07:53:18 2013
@@ -25,6 +25,7 @@ import java.util.List;
 
 import org.apache.lucene.search.SearcherManager; // javadocs
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.NoSuchDirectoryException;
 
 /** DirectoryReader is an implementation of {@link CompositeReader}
  that can read indexes in a {@link Directory}. 
@@ -313,17 +314,45 @@ public abstract class DirectoryReader ex
   }
   
   /**
-   * Returns <code>true</code> if an index exists at the specified directory.
+   * Returns <code>true</code> if an index likely exists at
+   * the specified directory.  Note that if a corrupt index
+   * exists, or if an index in the process of committing 
    * @param  directory the directory to check for an index
    * @return <code>true</code> if an index exists; <code>false</code> otherwise
    */
-  public static boolean indexExists(Directory directory) {
+  public static boolean indexExists(Directory directory) throws IOException {
+    // LUCENE-2812, LUCENE-2727, LUCENE-4738: this logic will
+    // return true in cases that should arguably be false,
+    // such as only IW.prepareCommit has been called, or a
+    // corrupt first commit, but it's too deadly to make
+    // this logic "smarter" and risk accidentally returning
+    // false due to various cases like file description
+    // exhaustion, access denied, etc., because in that
+    // case IndexWriter may delete the entire index.  It's
+    // safer to err towards "index exists" than try to be
+    // smart about detecting not-yet-fully-committed or
+    // corrupt indices.  This means that IndexWriter will
+    // throw an exception on such indices and the app must
+    // resolve the situation manually:
+    String[] files;
     try {
-      new SegmentInfos().read(directory);
-      return true;
-    } catch (IOException ioe) {
+      files = directory.listAll();
+    } catch (NoSuchDirectoryException nsde) {
+      // Directory does not exist --> no index exists
       return false;
     }
+
+    // Defensive: maybe a Directory impl returns null
+    // instead of throwing NoSuchDirectoryException:
+    if (files != null) {
+      String prefix = IndexFileNames.SEGMENTS + "_";
+      for(String file : files) {
+        if (file.startsWith(prefix) || file.equals(IndexFileNames.SEGMENTS_GEN)) {
+          return true;
+        }
+      }
+    }
+    return false;
   }
 
   /**

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java Thu May 30 07:53:18 2013
@@ -213,7 +213,7 @@ final class DocFieldProcessor extends Do
     // sort the subset of fields that have vectors
     // enabled; we could save [small amount of] CPU
     // here.
-    ArrayUtil.quickSort(fields, 0, fieldCount, fieldsComp);
+    ArrayUtil.introSort(fields, 0, fieldCount, fieldsComp);
     for(int i=0;i<fieldCount;i++) {
       final DocFieldProcessorPerField perField = fields[i];
       perField.consumer.processFields(perField.fields, perField.fieldCount);

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocInverterPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocInverterPerField.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocInverterPerField.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocInverterPerField.java Thu May 30 07:53:18 2013
@@ -119,10 +119,10 @@ final class DocInverterPerField extends 
 
               final int posIncr = posIncrAttribute.getPositionIncrement();
               if (posIncr < 0) {
-                throw new IllegalArgumentException("position increment must be >=0 (got " + posIncr + ")");
+                throw new IllegalArgumentException("position increment must be >=0 (got " + posIncr + ") for field '" + field.name() + "'");
               }
               if (fieldState.position == 0 && posIncr == 0) {
-                throw new IllegalArgumentException("first position increment must be > 0 (got 0)");
+                throw new IllegalArgumentException("first position increment must be > 0 (got 0) for field '" + field.name() + "'");
               }
               int position = fieldState.position + posIncr;
               if (position > 0) {
@@ -145,11 +145,11 @@ final class DocInverterPerField extends 
                 int endOffset = fieldState.offset + offsetAttribute.endOffset();
                 if (startOffset < 0 || endOffset < startOffset) {
                   throw new IllegalArgumentException("startOffset must be non-negative, and endOffset must be >= startOffset, "
-                      + "startOffset=" + startOffset + ",endOffset=" + endOffset);
+                      + "startOffset=" + startOffset + ",endOffset=" + endOffset + " for field '" + field.name() + "'");
                 }
                 if (startOffset < lastStartOffset) {
                   throw new IllegalArgumentException("offsets must not go backwards startOffset=" 
-                       + startOffset + " is < lastStartOffset=" + lastStartOffset);
+                       + startOffset + " is < lastStartOffset=" + lastStartOffset + " for field '" + field.name() + "'");
                 }
                 lastStartOffset = startOffset;
               }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java Thu May 30 07:53:18 2013
@@ -24,6 +24,7 @@ import java.util.Comparator;
 import java.util.List;
 
 import org.apache.lucene.codecs.PostingsFormat; // javadocs
+import org.apache.lucene.index.TermsEnum.SeekStatus;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -632,7 +633,9 @@ public class DocTermOrds {
 
     @Override
     public BytesRef next() throws IOException {
-      ord++;
+      if (++ord < 0) {
+        ord = 0;
+      }
       if (termsEnum.next() == null) {
         term = null;
         return null;
@@ -763,16 +766,17 @@ public class DocTermOrds {
   }
   
   /** Returns a SortedSetDocValues view of this instance */
-  public SortedSetDocValues iterator(TermsEnum termsEnum) throws IOException {
+  public SortedSetDocValues iterator(AtomicReader reader) throws IOException {
     if (isEmpty()) {
       return SortedSetDocValues.EMPTY;
     } else {
-      return new Iterator(termsEnum);
+      return new Iterator(reader);
     }
   }
   
   private class Iterator extends SortedSetDocValues {
-    final TermsEnum te;
+    final AtomicReader reader;
+    final TermsEnum te;  // used internally for lookupOrd() and lookupTerm()
     // currently we read 5 at a time (using the logic of the old iterator)
     final int buffer[] = new int[5];
     int bufferUpto;
@@ -782,8 +786,9 @@ public class DocTermOrds {
     private int upto;
     private byte[] arr;
     
-    Iterator(TermsEnum te) {
-      this.te = te;
+    Iterator(AtomicReader reader) throws IOException {
+      this.reader = reader;
+      this.te = termsEnum();
     }
     
     @Override
@@ -880,5 +885,27 @@ public class DocTermOrds {
     public long getValueCount() {
       return numTerms();
     }
+
+    @Override
+    public long lookupTerm(BytesRef key) {
+      try {
+        if (te.seekCeil(key) == SeekStatus.FOUND) {
+          return te.ord();
+        } else {
+          return -te.ord()-1;
+        }
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+    
+    @Override
+    public TermsEnum termsEnum() {    
+      try {
+        return getOrdTermsEnum(reader);
+      } catch (IOException e) {
+        throw new RuntimeException();
+      }
+    }
   }
 }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java Thu May 30 07:53:18 2013
@@ -29,6 +29,7 @@ import org.apache.lucene.index.Documents
 import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain;
 import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
 import org.apache.lucene.index.FieldInfos.FieldNumbers;
+import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.store.AlreadyClosedException;
@@ -240,6 +241,63 @@ final class DocumentsWriter {
       }
     }
   }
+  
+  synchronized void lockAndAbortAll() {
+    assert indexWriter.holdsFullFlushLock();
+    if (infoStream.isEnabled("DW")) {
+      infoStream.message("DW", "lockAndAbortAll");
+    }
+    boolean success = false;
+    try {
+      deleteQueue.clear();
+      final int limit = perThreadPool.getMaxThreadStates();
+      for (int i = 0; i < limit; i++) {
+        final ThreadState perThread = perThreadPool.getThreadState(i);
+        perThread.lock();
+        if (perThread.isActive()) { // we might be closed or 
+          try {
+            perThread.dwpt.abort();
+          } finally {
+            perThread.dwpt.checkAndResetHasAborted();
+            flushControl.doOnAbort(perThread);
+          }
+        }
+      }
+      deleteQueue.clear();
+      flushControl.abortPendingFlushes();
+      flushControl.waitForFlush();
+      success = true;
+    } finally {
+      if (infoStream.isEnabled("DW")) {
+        infoStream.message("DW", "finished lockAndAbortAll success=" + success);
+      }
+      if (!success) {
+        // if something happens here we unlock all states again
+        unlockAllAfterAbortAll();
+      }
+    }
+  }
+  
+  final synchronized void unlockAllAfterAbortAll() {
+    assert indexWriter.holdsFullFlushLock();
+    if (infoStream.isEnabled("DW")) {
+      infoStream.message("DW", "unlockAll");
+    }
+    final int limit = perThreadPool.getMaxThreadStates();
+    for (int i = 0; i < limit; i++) {
+      try {
+        final ThreadState perThread = perThreadPool.getThreadState(i);
+        if (perThread.isHeldByCurrentThread()) {
+          perThread.unlock();
+        }
+      } catch(Throwable e) {
+        if (infoStream.isEnabled("DW")) {
+          infoStream.message("DW", "unlockAll: could not unlock state: " + i + " msg:" + e.getMessage());
+        }
+        // ignore & keep on unlocking
+      }
+    }
+  }
 
   boolean anyChanges() {
     if (infoStream.isEnabled("DW")) {

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java Thu May 30 07:53:18 2013
@@ -240,6 +240,7 @@ final class DocumentsWriterFlushControl 
   }
   
   public synchronized void waitForFlush() {
+    assert !Thread.holdsLock(this.documentsWriter.indexWriter) : "IW lock should never be hold when waiting on flush";
     while (flushingWriters.size() != 0) {
       try {
         this.wait();
@@ -606,9 +607,10 @@ final class DocumentsWriterFlushControl 
       for (DocumentsWriterPerThread dwpt : flushQueue) {
         try {
           dwpt.abort();
-          doAfterFlush(dwpt);
         } catch (Throwable ex) {
           // ignore - keep on aborting the flush queue
+        } finally {
+          doAfterFlush(dwpt);
         }
       }
       for (BlockedFlush blockedFlush : blockedFlushes) {
@@ -616,9 +618,10 @@ final class DocumentsWriterFlushControl 
           flushingWriters
               .put(blockedFlush.dwpt, Long.valueOf(blockedFlush.bytes));
           blockedFlush.dwpt.abort();
-          doAfterFlush(blockedFlush.dwpt);
         } catch (Throwable ex) {
           // ignore - keep on aborting the blocked queue
+        } finally {
+          doAfterFlush(blockedFlush.dwpt);
         }
       }
     } finally {

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Thu May 30 07:53:18 2013
@@ -555,7 +555,7 @@ class DocumentsWriterPerThread {
 
     SegmentInfoPerCommit newSegment = flushedSegment.segmentInfo;
 
-    IndexWriter.setDiagnostics(newSegment.info, "flush");
+    IndexWriter.setDiagnostics(newSegment.info, IndexWriter.SOURCE_FLUSH);
     
     IOContext context = new IOContext(new FlushInfo(newSegment.info.getDocCount(), newSegment.sizeInBytes()));
 

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java Thu May 30 07:53:18 2013
@@ -274,7 +274,6 @@ abstract class DocumentsWriterPerThreadP
    *         given ord.
    */
   ThreadState getThreadState(int ord) {
-    assert ord < numThreadStatesActive;
     return threadStates[ord];
   }
 

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java Thu May 30 07:53:18 2013
@@ -17,15 +17,16 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import org.apache.lucene.util.AttributeSource;
-import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.automaton.CompiledAutomaton;
-
 import java.io.IOException;
 import java.util.Comparator;
 import java.util.Iterator;
 
+import org.apache.lucene.search.CachingWrapperFilter;
+import org.apache.lucene.search.FieldCache;
+import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+
 /**  A <code>FilterAtomicReader</code> contains another AtomicReader, which it
  * uses as its basic source of data, possibly transforming the data along the
  * way or providing additional functionality. The class
@@ -34,6 +35,15 @@ import java.util.Iterator;
  * contained index reader. Subclasses of <code>FilterAtomicReader</code> may
  * further override some of these methods and may also provide additional
  * methods and fields.
+ * <p><b>NOTE</b>: If you override {@link #getLiveDocs()}, you will likely need
+ * to override {@link #numDocs()} as well and vice-versa.
+ * <p><b>NOTE</b>: If this {@link FilterAtomicReader} does not change the
+ * content the contained reader, you could consider overriding
+ * {@link #getCoreCacheKey()} so that {@link FieldCache} and
+ * {@link CachingWrapperFilter} share the same entries for this atomic reader
+ * and the wrapped one. {@link #getCombinedCoreAndDeletesKey()} could be
+ * overridden as well if the {@link #getLiveDocs() live docs} are not changed
+ * either.
  */
 public class FilterAtomicReader extends AtomicReader {
 
@@ -67,8 +77,11 @@ public class FilterAtomicReader extends 
     }
   }
 
-  /** Base class for filtering {@link Terms}
-   *  implementations. */
+  /** Base class for filtering {@link Terms} implementations.
+   * <p><b>NOTE</b>: If the order of terms and documents is not changed, and if
+   * these terms are going to be intersected with automata, you could consider
+   * overriding {@link #intersect} for better performance.
+   */
   public static class FilterTerms extends Terms {
     /** The underlying Terms instance. */
     protected final Terms in;
@@ -85,7 +98,7 @@ public class FilterAtomicReader extends 
     public TermsEnum iterator(TermsEnum reuse) throws IOException {
       return in.iterator(reuse);
     }
-
+    
     @Override
     public Comparator<BytesRef> getComparator() {
       return in.getComparator();
@@ -110,11 +123,6 @@ public class FilterAtomicReader extends 
     public int getDocCount() throws IOException {
       return in.getDocCount();
     }
-    
-    @Override
-    public TermsEnum intersect(CompiledAutomaton automaton, BytesRef bytes) throws java.io.IOException {
-      return in.intersect(automaton, bytes);
-    }
 
     @Override
     public boolean hasOffsets() {
@@ -144,8 +152,8 @@ public class FilterAtomicReader extends 
     public FilterTermsEnum(TermsEnum in) { this.in = in; }
 
     @Override
-    public boolean seekExact(BytesRef text, boolean useCache) throws IOException {
-      return in.seekExact(text, useCache);
+    public AttributeSource attributes() {
+      return in.attributes();
     }
 
     @Override
@@ -197,21 +205,6 @@ public class FilterAtomicReader extends 
     public Comparator<BytesRef> getComparator() {
       return in.getComparator();
     }
-
-    @Override
-    public void seekExact(BytesRef term, TermState state) throws IOException {
-      in.seekExact(term, state);
-    }
-
-    @Override
-    public TermState termState() throws IOException {
-      return in.termState();
-    }
-    
-    @Override
-    public AttributeSource attributes() {
-      return in.attributes();
-    }
   }
 
   /** Base class for filtering {@link DocsEnum} implementations. */
@@ -228,6 +221,11 @@ public class FilterAtomicReader extends 
     }
 
     @Override
+    public AttributeSource attributes() {
+      return in.attributes();
+    }
+
+    @Override
     public int docID() {
       return in.docID();
     }
@@ -246,10 +244,10 @@ public class FilterAtomicReader extends 
     public int advance(int target) throws IOException {
       return in.advance(target);
     }
-    
+
     @Override
-    public AttributeSource attributes() {
-      return in.attributes();
+    public long cost() {
+      return in.cost();
     }
   }
 
@@ -267,6 +265,11 @@ public class FilterAtomicReader extends 
     }
 
     @Override
+    public AttributeSource attributes() {
+      return in.attributes();
+    }
+
+    @Override
     public int docID() {
       return in.docID();
     }
@@ -307,8 +310,8 @@ public class FilterAtomicReader extends 
     }
     
     @Override
-    public AttributeSource attributes() {
-      return in.attributes();
+    public long cost() {
+      return in.cost();
     }
   }
 
@@ -363,12 +366,6 @@ public class FilterAtomicReader extends 
   }
 
   @Override
-  public boolean hasDeletions() {
-    ensureOpen();
-    return in.hasDeletions();
-  }
-
-  @Override
   protected void doClose() throws IOException {
     in.close();
   }
@@ -379,24 +376,6 @@ public class FilterAtomicReader extends 
     return in.fields();
   }
 
-  /** {@inheritDoc}
-   * <p>If the subclass of FilteredIndexReader modifies the
-   *  contents (but not liveDocs) of the index, you must override this
-   *  method to provide a different key. */
-  @Override
-  public Object getCoreCacheKey() {
-    return in.getCoreCacheKey();
-  }
-
-  /** {@inheritDoc}
-   * <p>If the subclass of FilteredIndexReader modifies the
-   *  liveDocs, you must override this
-   *  method to provide a different key. */
-  @Override
-  public Object getCombinedCoreAndDeletesKey() {
-    return in.getCombinedCoreAndDeletesKey();
-  }
-
   @Override
   public String toString() {
     final StringBuilder buffer = new StringBuilder("FilterAtomicReader(");
@@ -434,4 +413,5 @@ public class FilterAtomicReader extends 
     ensureOpen();
     return in.getNormValues(field);
   }
+
 }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java Thu May 30 07:53:18 2013
@@ -54,7 +54,7 @@ final class FreqProxTermsWriter extends 
     final int numAllFields = allFields.size();
 
     // Sort by field name
-    CollectionUtil.quickSort(allFields);
+    CollectionUtil.introSort(allFields);
 
     final FieldsConsumer consumer = state.segmentInfo.getCodec().postingsFormat().fieldsConsumer(state);
 

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java Thu May 30 07:53:18 2013
@@ -56,7 +56,7 @@ class FrozenBufferedDeletes {
     assert !isSegmentPrivate || deletes.terms.size() == 0 : "segment private package should only have del queries"; 
     Term termsArray[] = deletes.terms.keySet().toArray(new Term[deletes.terms.size()]);
     termCount = termsArray.length;
-    ArrayUtil.mergeSort(termsArray);
+    ArrayUtil.timSort(termsArray);
     PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder();
     for (Term term : termsArray) {
       builder.add(term);

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexCommit.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexCommit.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexCommit.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexCommit.java Thu May 30 07:53:18 2013
@@ -119,12 +119,6 @@ public abstract class IndexCommit implem
 
     long gen = getGeneration();
     long comgen = commit.getGeneration();
-    if (gen < comgen) {
-      return -1;
-    } else if (gen > comgen) {
-      return 1;
-    } else {
-      return 0;
-    }
+    return Long.compare(gen, comgen);
   }
 }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexDeletionPolicy.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexDeletionPolicy.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexDeletionPolicy.java Thu May 30 07:53:18 2013
@@ -48,7 +48,10 @@ import java.io.IOException;
  * for details.</p>
  */
 
-public interface IndexDeletionPolicy {
+public abstract class IndexDeletionPolicy implements Cloneable {
+
+  /** Sole constructor, typically called by sub-classes constructors. */
+  protected IndexDeletionPolicy() {}
 
   /**
    * <p>This is called once when a writer is first
@@ -70,7 +73,7 @@ public interface IndexDeletionPolicy {
    * {@link IndexCommit point-in-time commits},
    *  sorted by age (the 0th one is the oldest commit).
    */
-  public void onInit(List<? extends IndexCommit> commits) throws IOException;
+  public abstract void onInit(List<? extends IndexCommit> commits) throws IOException;
 
   /**
    * <p>This is called each time the writer completed a commit.
@@ -94,5 +97,15 @@ public interface IndexDeletionPolicy {
    * @param commits List of {@link IndexCommit},
    *  sorted by age (the 0th one is the oldest commit).
    */
-  public void onCommit(List<? extends IndexCommit> commits) throws IOException;
+  public abstract void onCommit(List<? extends IndexCommit> commits) throws IOException;
+
+  @Override
+  public IndexDeletionPolicy clone() {
+    try {
+      return (IndexDeletionPolicy) super.clone();
+    } catch (CloneNotSupportedException e) {
+      throw new Error(e);
+    }
+  }
+
 }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java Thu May 30 07:53:18 2013
@@ -123,7 +123,7 @@ final class IndexFileDeleter implements 
    * @throws IOException if there is a low-level IO error
    */
   public IndexFileDeleter(Directory directory, IndexDeletionPolicy policy, SegmentInfos segmentInfos,
-                          InfoStream infoStream, IndexWriter writer) throws IOException {
+                          InfoStream infoStream, IndexWriter writer, boolean initialIndexExists) throws IOException {
     this.infoStream = infoStream;
     this.writer = writer;
 
@@ -209,7 +209,7 @@ final class IndexFileDeleter implements 
       }
     }
 
-    if (currentCommitPoint == null && currentSegmentsFile != null) {
+    if (currentCommitPoint == null && currentSegmentsFile != null && initialIndexExists) {
       // We did not in fact see the segments_N file
       // corresponding to the segmentInfos that was passed
       // in.  Yet, it must exist, because our caller holds
@@ -221,7 +221,7 @@ final class IndexFileDeleter implements 
       try {
         sis.read(directory, currentSegmentsFile);
       } catch (IOException e) {
-        throw new CorruptIndexException("failed to locate current segments_N file");
+        throw new CorruptIndexException("failed to locate current segments_N file \"" + currentSegmentsFile + "\"");
       }
       if (infoStream.isEnabled("IFD")) {
         infoStream.message("IFD", "forced open of current segments file " + segmentInfos.getSegmentsFileName());
@@ -232,7 +232,7 @@ final class IndexFileDeleter implements 
     }
 
     // We keep commits list in sorted order (oldest to newest):
-    CollectionUtil.mergeSort(commits);
+    CollectionUtil.timSort(commits);
 
     // Now delete anything with ref count at 0.  These are
     // presumably abandoned files eg due to crash of
@@ -250,9 +250,7 @@ final class IndexFileDeleter implements 
 
     // Finally, give policy a chance to remove things on
     // startup:
-    if (currentSegmentsFile != null) {
-      policy.onInit(commits);
-    }
+    policy.onInit(commits);
 
     // Always protect the incoming segmentInfos since
     // sometime it may not be the most recent commit

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexFileNames.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexFileNames.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexFileNames.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexFileNames.java Thu May 30 07:53:18 2013
@@ -199,7 +199,10 @@ public final class IndexFileNames {
     return filename;
   }  
 
-  // All files created by codecs much match this pattern (we
-  // check this in SegmentInfo.java):
-  static final Pattern CODEC_FILE_PATTERN = Pattern.compile("_[a-z0-9]+(_.*)?\\..*");
+  /**
+   * All files created by codecs much match this pattern (checked in
+   * SegmentInfo).
+   */
+  public static final Pattern CODEC_FILE_PATTERN = Pattern.compile("_[a-z0-9]+(_.*)?\\..*");
+  
 }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexReader.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexReader.java Thu May 30 07:53:18 2013
@@ -22,15 +22,14 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.LinkedHashSet;
 import java.util.List;
-import java.util.WeakHashMap;
 import java.util.Set;
+import java.util.WeakHashMap;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.lucene.document.Document;
 import org.apache.lucene.document.DocumentStoredFieldVisitor;
-import org.apache.lucene.search.SearcherManager; // javadocs
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.util.Bits;
+// javadocs
 
 /** IndexReader is an abstract class, providing an interface for accessing an
  index.  Search of an index is done entirely through this abstract interface,
@@ -179,7 +178,7 @@ public abstract class IndexReader implem
    * and returns <code>true</code> iff the refCount was
    * successfully incremented, otherwise <code>false</code>.
    * If this method returns <code>false</code> the reader is either
-   * already closed or is currently been closed. Either way this
+   * already closed or is currently being closed. Either way this
    * reader instance shouldn't be used by an application unless
    * <code>true</code> is returned.
    * <p>
@@ -361,8 +360,12 @@ public abstract class IndexReader implem
     return visitor.getDocument();
   }
 
-  /** Returns true if any documents have been deleted */
-  public abstract boolean hasDeletions();
+  /** Returns true if any documents have been deleted. Implementers should
+   *  consider overriding this method if {@link #maxDoc()} or {@link #numDocs()}
+   *  are not constant-time operations. */
+  public boolean hasDeletions() {
+    return numDeletedDocs() > 0;
+  }
 
   /**
    * Closes files associated with this index.
@@ -415,7 +418,7 @@ public abstract class IndexReader implem
    * it again.
    * This key must not have equals()/hashCode() methods, so &quot;equals&quot; means &quot;identical&quot;. */
   public Object getCoreCacheKey() {
-    // Don't can ensureOpen since FC calls this (to evict)
+    // Don't call ensureOpen since FC calls this (to evict)
     // on close
     return this;
   }
@@ -424,7 +427,7 @@ public abstract class IndexReader implem
    * so FieldCache/CachingWrapperFilter can find it again.
    * This key must not have equals()/hashCode() methods, so &quot;equals&quot; means &quot;identical&quot;. */
   public Object getCombinedCoreAndDeletesKey() {
-    // Don't can ensureOpen since FC calls this (to evict)
+    // Don't call ensureOpen since FC calls this (to evict)
     // on close
     return this;
   }
@@ -438,12 +441,40 @@ public abstract class IndexReader implem
    */
   public abstract int docFreq(Term term) throws IOException;
   
-  /** Returns the number of documents containing the term
-   * <code>term</code>.  This method returns 0 if the term or
-   * field does not exists, or -1 if the Codec does not support
-   * the measure.  This method does not take into account deleted 
-   * documents that have not yet been merged away.
-   * @see TermsEnum#totalTermFreq() 
+  /**
+   * Returns the total number of occurrences of {@code term} across all
+   * documents (the sum of the freq() for each doc that has this term). This
+   * will be -1 if the codec doesn't support this measure. Note that, like other
+   * term measures, this measure does not take deleted documents into account.
    */
   public abstract long totalTermFreq(Term term) throws IOException;
+  
+  /**
+   * Returns the sum of {@link TermsEnum#docFreq()} for all terms in this field,
+   * or -1 if this measure isn't stored by the codec. Note that, just like other
+   * term measures, this measure does not take deleted documents into account.
+   * 
+   * @see Terms#getSumDocFreq()
+   */
+  public abstract long getSumDocFreq(String field) throws IOException;
+  
+  /**
+   * Returns the number of documents that have at least one term for this field,
+   * or -1 if this measure isn't stored by the codec. Note that, just like other
+   * term measures, this measure does not take deleted documents into account.
+   * 
+   * @see Terms#getDocCount()
+   */
+  public abstract int getDocCount(String field) throws IOException;
+
+  /**
+   * Returns the sum of {@link TermsEnum#totalTermFreq} for all terms in this
+   * field, or -1 if this measure isn't stored by the codec (or if this fields
+   * omits term freq and positions). Note that, just like other term measures,
+   * this measure does not take deleted documents into account.
+   * 
+   * @see Terms#getSumTotalTermFreq()
+   */
+  public abstract long getSumTotalTermFreq(String field) throws IOException;
+
 }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Thu May 30 07:53:18 2013
@@ -185,13 +185,21 @@ import org.apache.lucene.util.ThreadInte
 public class IndexWriter implements Closeable, TwoPhaseCommit {
   
   private static final int UNBOUNDED_MAX_MERGE_SEGMENTS = -1;
-
   
   /**
    * Name of the write lock in the index.
    */
   public static final String WRITE_LOCK_NAME = "write.lock";
 
+  /** Key for the source of a segment in the {@link SegmentInfo#getDiagnostics() diagnostics}. */
+  public static final String SOURCE = "source";
+  /** Source of a segment which results from a merge of other segments. */
+  public static final String SOURCE_MERGE = "merge";
+  /** Source of a segment which results from a flush. */
+  public static final String SOURCE_FLUSH = "flush";
+  /** Source of a segment which results from a call to {@link #addIndexes(IndexReader...)}. */
+  public static final String SOURCE_ADDINDEXES_READERS = "addIndexes(IndexReader...)";
+
   /**
    * Absolute hard maximum length for a term, in bytes once
    * encoded as UTF8.  If a term arrives from the analyzer
@@ -360,7 +368,7 @@ public class IndexWriter implements Clos
           }
           success = true;
           // Prevent segmentInfos from changing while opening the
-          // reader; in theory we could do similar retry logic,
+          // reader; in theory we could instead do similar retry logic,
           // just like we do when loading segments_N
           synchronized(this) {
             maybeApplyDeletes(applyAllDeletes);
@@ -427,6 +435,16 @@ public class IndexWriter implements Clos
       }
     }
 
+    public synchronized boolean anyPendingDeletes() {
+      for(ReadersAndLiveDocs rld : readerMap.values()) {
+        if (rld.getPendingDeleteCount() != 0) {
+          return true;
+        }
+      }
+
+      return false;
+    }
+
     public synchronized void release(ReadersAndLiveDocs rld) throws IOException {
 
       // Matches incRef in get:
@@ -545,8 +563,21 @@ public class IndexWriter implements Clos
         rld.incRef();
       }
 
+      assert noDups();
+
       return rld;
     }
+
+    // Make sure that every segment appears only once in the
+    // pool:
+    private boolean noDups() {
+      Set<String> seen = new HashSet<String>();
+      for(SegmentInfoPerCommit info : readerMap.keySet()) {
+        assert !seen.contains(info.info.name);
+        seen.add(info.info.name);
+      }
+      return true;
+    }
   }
 
   /**
@@ -651,6 +682,8 @@ public class IndexWriter implements Clos
       // IndexFormatTooOldException.
       segmentInfos = new SegmentInfos();
 
+      boolean initialIndexExists = true;
+
       if (create) {
         // Try to read first.  This is to allow create
         // against an index that's currently open for
@@ -661,12 +694,12 @@ public class IndexWriter implements Clos
           segmentInfos.clear();
         } catch (IOException e) {
           // Likely this means it's a fresh directory
+          initialIndexExists = false;
         }
 
         // Record that we have a change (zero out all
         // segments) pending:
-        changeCount++;
-        segmentInfos.changed();
+        changed();
       } else {
         segmentInfos.read(directory);
 
@@ -682,8 +715,7 @@ public class IndexWriter implements Clos
           SegmentInfos oldInfos = new SegmentInfos();
           oldInfos.read(directory, commit.getSegmentsFileName());
           segmentInfos.replace(oldInfos);
-          changeCount++;
-          segmentInfos.changed();
+          changed();
           if (infoStream.isEnabled("IW")) {
             infoStream.message("IW", "init: loaded commit \"" + commit.getSegmentsFileName() + "\"");
           }
@@ -701,7 +733,8 @@ public class IndexWriter implements Clos
       synchronized(this) {
         deleter = new IndexFileDeleter(directory,
                                        config.getIndexDeletionPolicy(),
-                                       segmentInfos, infoStream, this);
+                                       segmentInfos, infoStream, this,
+                                       initialIndexExists);
       }
 
       if (deleter.startingCommitDeleted) {
@@ -709,8 +742,7 @@ public class IndexWriter implements Clos
         // We have to mark ourself as changed so that if we
         // are closed w/o any further changes we write a new
         // segments_N file.
-        changeCount++;
-        segmentInfos.changed();
+        changed();
       }
 
       if (infoStream.isEnabled("IW")) {
@@ -925,7 +957,7 @@ public class IndexWriter implements Clos
         if (doFlush) {
           flush(waitForMerges, true);
         } else {
-          docWriter.abort(); // already closed
+          docWriter.abort(); // already closed -- never sync on IW 
         }
         
       } finally {
@@ -1073,6 +1105,9 @@ public class IndexWriter implements Clos
     if (docWriter.anyDeletions()) {
       return true;
     }
+    if (readerPool.anyPendingDeletes()) {
+      return true;
+    }
     for (final SegmentInfoPerCommit info : segmentInfos) {
       if (info.hasDeletions()) {
         return true;
@@ -1342,7 +1377,7 @@ public class IndexWriter implements Clos
 
             // Must bump changeCount so if no other changes
             // happened, we still commit this change:
-            changeCount++;
+            changed();
           }
           //System.out.println("  yes " + info.info.name + " " + docID);
           return true;
@@ -1971,7 +2006,7 @@ public class IndexWriter implements Clos
 
       bufferedDeletesStream.clear();
       docWriter.close(); // mark it as closed first to prevent subsequent indexing actions/flushes 
-      docWriter.abort();
+      docWriter.abort(); // don't sync on IW here
       synchronized(this) {
 
         if (pendingCommit != null) {
@@ -2031,7 +2066,13 @@ public class IndexWriter implements Clos
    *    visible until a {@link #commit()} has been called. This method
    *    can be rolled back using {@link #rollback()}.</p>
    *
-   * <p>NOTE: this method is much faster than using deleteDocuments( new MatchAllDocsQuery() ).</p>
+   * <p>NOTE: this method is much faster than using deleteDocuments( new MatchAllDocsQuery() ). 
+   *    Yet, this method also has different semantics compared to {@link #deleteDocuments(Query)} 
+   *    / {@link #deleteDocuments(Query...)} since internal data-structures are cleared as well 
+   *    as all segment information is forcefully dropped anti-viral semantics like omitting norms
+   *    are reset or doc value types are cleared. Essentially a call to {@link #deleteAll()} is equivalent
+   *    to creating a new {@link IndexWriter} with {@link OpenMode#CREATE} which a delete query only marks
+   *    documents as deleted.</p>
    *
    * <p>NOTE: this method will forcefully abort all merges
    *    in progress.  If other threads are running {@link
@@ -2039,40 +2080,58 @@ public class IndexWriter implements Clos
    *    {@link #forceMergeDeletes} methods, they may receive
    *    {@link MergePolicy.MergeAbortedException}s.
    */
-  public synchronized void deleteAll() throws IOException {
+  public void deleteAll() throws IOException {
     ensureOpen();
+    // Remove any buffered docs
     boolean success = false;
-    try {
-
-      // Abort any running merges
-      finishMerges(false);
-
-      // Remove any buffered docs
-      docWriter.abort();
-
-      // Remove all segments
-      segmentInfos.clear();
-
-      // Ask deleter to locate unreferenced files & remove them:
-      deleter.checkpoint(segmentInfos, false);
-      deleter.refresh();
-
-      globalFieldNumberMap.clear();
-
-      // Don't bother saving any changes in our segmentInfos
-      readerPool.dropAll(false);
-
-      // Mark that the index has changed
-      ++changeCount;
-      segmentInfos.changed();
-      success = true;
-    } catch (OutOfMemoryError oom) {
-      handleOOM(oom, "deleteAll");
-    } finally {
-      if (!success) {
-        if (infoStream.isEnabled("IW")) {
-          infoStream.message("IW", "hit exception during deleteAll");
+    /* hold the full flush lock to prevent concurrency commits / NRT reopens to
+     * get in our way and do unnecessary work. -- if we don't lock this here we might
+     * get in trouble if */
+    synchronized (fullFlushLock) { 
+        /*
+         * We first abort and trash everything we have in-memory
+         * and keep the thread-states locked, the lockAndAbortAll operation
+         * also guarantees "point in time semantics" ie. the checkpoint that we need in terms
+         * of logical happens-before relationship in the DW. So we do
+         * abort all in memory structures 
+         * We also drop global field numbering before during abort to make
+         * sure it's just like a fresh index.
+         */
+      try {
+        docWriter.lockAndAbortAll();
+        synchronized (this) {
+          try {
+            // Abort any running merges
+            finishMerges(false);
+            // Remove all segments
+            segmentInfos.clear();
+            // Ask deleter to locate unreferenced files & remove them:
+            deleter.checkpoint(segmentInfos, false);
+            /* don't refresh the deleter here since there might
+             * be concurrent indexing requests coming in opening
+             * files on the directory after we called DW#abort()
+             * if we do so these indexing requests might hit FNF exceptions.
+             * We will remove the files incrementally as we go...
+             */
+            // Don't bother saving any changes in our segmentInfos
+            readerPool.dropAll(false);
+            // Mark that the index has changed
+            ++changeCount;
+            segmentInfos.changed();
+            globalFieldNumberMap.clear();
+            success = true;
+          } catch (OutOfMemoryError oom) {
+            handleOOM(oom, "deleteAll");
+          } finally {
+            if (!success) {
+              if (infoStream.isEnabled("IW")) {
+                infoStream.message("IW", "hit exception during deleteAll");
+              }
+            }
+          }
         }
+      } finally {
+        docWriter.unlockAllAfterAbortAll();
       }
     }
   }
@@ -2159,9 +2218,14 @@ public class IndexWriter implements Clos
    * index directory.
    */
   synchronized void checkpoint() throws IOException {
+    changed();
+    deleter.checkpoint(segmentInfos, false);
+  }
+
+  /** Called internally if any index state has changed. */
+  synchronized void changed() {
     changeCount++;
     segmentInfos.changed();
-    deleter.checkpoint(segmentInfos, false);
   }
 
   synchronized void publishFrozenDeletes(FrozenBufferedDeletes packet) {
@@ -2395,8 +2459,12 @@ public class IndexWriter implements Clos
       flush(false, true);
 
       String mergedName = newSegmentName();
+      final List<AtomicReader> mergeReaders = new ArrayList<AtomicReader>();
       for (IndexReader indexReader : readers) {
         numDocs += indexReader.numDocs();
+        for (AtomicReaderContext ctx : indexReader.leaves()) {
+          mergeReaders.add(ctx.reader());
+        }
       }
       final IOContext context = new IOContext(new MergeInfo(numDocs, -1, true, -1));
 
@@ -2407,13 +2475,9 @@ public class IndexWriter implements Clos
       SegmentInfo info = new SegmentInfo(directory, Constants.LUCENE_MAIN_VERSION, mergedName, -1,
                                          false, codec, null, null);
 
-      SegmentMerger merger = new SegmentMerger(info, infoStream, trackingDir, config.getTermIndexInterval(),
+      SegmentMerger merger = new SegmentMerger(mergeReaders, info, infoStream, trackingDir, config.getTermIndexInterval(),
                                                MergeState.CheckAbort.NONE, globalFieldNumberMap, context);
 
-      for (IndexReader reader : readers) {    // add new indexes
-        merger.add(reader);
-      }
-
       MergeState mergeState;
       boolean success = false;
       try {
@@ -2432,7 +2496,7 @@ public class IndexWriter implements Clos
       info.setFiles(new HashSet<String>(trackingDir.getCreatedFiles()));
       trackingDir.getCreatedFiles().clear();
                                          
-      setDiagnostics(info, "addIndexes(IndexReader...)");
+      setDiagnostics(info, SOURCE_ADDINDEXES_READERS);
 
       boolean useCompoundFile;
       synchronized(this) { // Guard segmentInfos
@@ -2827,6 +2891,11 @@ public class IndexWriter implements Clos
   // Ensures only one flush() is actually flushing segments
   // at a time:
   private final Object fullFlushLock = new Object();
+  
+  // for assert
+  boolean holdsFullFlushLock() {
+    return Thread.holdsLock(fullFlushLock);
+  }
 
   /**
    * Flush all in-memory buffered updates (adds and deletes)
@@ -2975,7 +3044,7 @@ public class IndexWriter implements Clos
    *  saves the resulting deletes file (incrementing the
    *  delete generation for merge.info).  If no deletes were
    *  flushed, no new deletes file is saved. */
-  synchronized private ReadersAndLiveDocs commitMergedDeletes(MergePolicy.OneMerge merge) throws IOException {
+  synchronized private ReadersAndLiveDocs commitMergedDeletes(MergePolicy.OneMerge merge, MergeState mergeState) throws IOException {
 
     assert testPoint("startCommitMergeDeletes");
 
@@ -2992,6 +3061,7 @@ public class IndexWriter implements Clos
 
     // Lazy init (only when we find a delete to carry over):
     ReadersAndLiveDocs mergedDeletes = null;
+    MergePolicy.DocMap docMap = null;
 
     for(int i=0; i < sourceSegments.size(); i++) {
       SegmentInfoPerCommit info = sourceSegments.get(i);
@@ -3037,8 +3107,10 @@ public class IndexWriter implements Clos
                 if (mergedDeletes == null) {
                   mergedDeletes = readerPool.get(merge.info, true);
                   mergedDeletes.initWritableLiveDocs();
+                  docMap = merge.getDocMap(mergeState);
+                  assert docMap.isConsistent(merge.info.info.getDocCount());
                 }
-                mergedDeletes.delete(docUpto);
+                mergedDeletes.delete(docMap.map(docUpto));
               }
               docUpto++;
             }
@@ -3055,8 +3127,10 @@ public class IndexWriter implements Clos
             if (mergedDeletes == null) {
               mergedDeletes = readerPool.get(merge.info, true);
               mergedDeletes.initWritableLiveDocs();
+              docMap = merge.getDocMap(mergeState);
+              assert docMap.isConsistent(merge.info.info.getDocCount());
             }
-            mergedDeletes.delete(docUpto);
+            mergedDeletes.delete(docMap.map(docUpto));
           }
           docUpto++;
         }
@@ -3081,7 +3155,7 @@ public class IndexWriter implements Clos
     return mergedDeletes;
   }
 
-  synchronized private boolean commitMerge(MergePolicy.OneMerge merge) throws IOException {
+  synchronized private boolean commitMerge(MergePolicy.OneMerge merge, MergeState mergeState) throws IOException {
 
     assert testPoint("startCommitMerge");
 
@@ -3109,7 +3183,7 @@ public class IndexWriter implements Clos
       return false;
     }
 
-    final ReadersAndLiveDocs mergedDeletes =  merge.info.info.getDocCount() == 0 ? null : commitMergedDeletes(merge);
+    final ReadersAndLiveDocs mergedDeletes =  merge.info.info.getDocCount() == 0 ? null : commitMergedDeletes(merge, mergeState);
 
     assert mergedDeletes == null || mergedDeletes.getPendingDeleteCount() != 0;
 
@@ -3450,16 +3524,15 @@ public class IndexWriter implements Clos
     // names.
     final String mergeSegmentName = newSegmentName();
     SegmentInfo si = new SegmentInfo(directory, Constants.LUCENE_MAIN_VERSION, mergeSegmentName, -1, false, codec, null, null);
-    merge.info = new SegmentInfoPerCommit(si, 0, -1L);
+    Map<String,String> details = new HashMap<String,String>();
+    details.put("mergeMaxNumSegments", "" + merge.maxNumSegments);
+    details.put("mergeFactor", Integer.toString(merge.segments.size()));
+    setDiagnostics(si, SOURCE_MERGE, details);
+    merge.setInfo(new SegmentInfoPerCommit(si, 0, -1L));
 
     // Lock order: IW -> BD
     bufferedDeletesStream.prune(segmentInfos);
 
-    Map<String,String> details = new HashMap<String,String>();
-    details.put("mergeMaxNumSegments", ""+merge.maxNumSegments);
-    details.put("mergeFactor", Integer.toString(merge.segments.size()));
-    setDiagnostics(si, "merge", details);
-
     if (infoStream.isEnabled("IW")) {
       infoStream.message("IW", "merge seg=" + merge.info.info.name + " " + segString(merge.segments));
     }
@@ -3561,9 +3634,6 @@ public class IndexWriter implements Clos
     final MergeState.CheckAbort checkAbort = new MergeState.CheckAbort(merge, directory);
     final TrackingDirectoryWrapper dirWrapper = new TrackingDirectoryWrapper(directory);
 
-    SegmentMerger merger = new SegmentMerger(merge.info.info, infoStream, dirWrapper, config.getTermIndexInterval(), checkAbort,
-                                             globalFieldNumberMap, context);
-
     if (infoStream.isEnabled("IW")) {
       infoStream.message("IW", "merging " + segString(merge.segments));
     }
@@ -3633,12 +3703,15 @@ public class IndexWriter implements Clos
 
         merge.readers.add(reader);
         assert delCount <= info.info.getDocCount(): "delCount=" + delCount + " info.docCount=" + info.info.getDocCount() + " rld.pendingDeleteCount=" + rld.getPendingDeleteCount() + " info.getDelCount()=" + info.getDelCount();
-        if (delCount < info.info.getDocCount()) {
-          merger.add(reader);
-        }
         segUpto++;
       }
 
+      // we pass merge.getMergeReaders() instead of merge.readers to allow the
+      // OneMerge to return a view over the actual segments to merge
+      final SegmentMerger merger = new SegmentMerger(merge.getMergeReaders(),
+          merge.info.info, infoStream, dirWrapper, config.getTermIndexInterval(),
+          checkAbort, globalFieldNumberMap, context);
+
       merge.checkAborted(directory);
 
       // This is where all the work happens:
@@ -3780,7 +3853,7 @@ public class IndexWriter implements Clos
 
       // Force READ context because we merge deletes onto
       // this reader:
-      if (!commitMerge(merge)) {
+      if (!commitMerge(merge, mergeState)) {
         // commitMerge will return false if this merge was aborted
         return 0;
       }
@@ -4097,8 +4170,7 @@ public class IndexWriter implements Clos
     //System.out.println("IW.nrtIsCurrent " + (infos.version == segmentInfos.version && !docWriter.anyChanges() && !bufferedDeletesStream.any()));
     ensureOpen();
     if (infoStream.isEnabled("IW")) {
-      infoStream.message("IW", "nrtIsCurrent: infoVersion matches: " + (infos.version == segmentInfos.version) + " DW changes: " + docWriter.anyChanges() + " BD changes: "+bufferedDeletesStream.any());
-
+      infoStream.message("IW", "nrtIsCurrent: infoVersion matches: " + (infos.version == segmentInfos.version) + "; DW changes: " + docWriter.anyChanges() + "; BD changes: "+ bufferedDeletesStream.any());
     }
     return infos.version == segmentInfos.version && !docWriter.anyChanges() && !bufferedDeletesStream.any();
   }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java Thu May 30 07:53:18 2013
@@ -23,7 +23,6 @@ import org.apache.lucene.analysis.Analyz
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain;
 import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
-import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util.PrintStreamInfoStream;
@@ -152,9 +151,14 @@ public final class IndexWriterConfig ext
       // Mostly shallow clone, but do a deepish clone of
       // certain objects that have state that cannot be shared
       // across IW instances:
+      clone.delPolicy = delPolicy.clone();
       clone.flushPolicy = flushPolicy.clone();
       clone.indexerThreadPool = indexerThreadPool.clone();
+      // we clone the infoStream because some impls might have state variables
+      // such as line numbers, message throughput, ...
+      clone.infoStream = infoStream.clone();
       clone.mergePolicy = mergePolicy.clone();
+      clone.mergeScheduler = mergeScheduler.clone();
       
       return clone;
     } catch (CloneNotSupportedException e) {
@@ -166,6 +170,9 @@ public final class IndexWriterConfig ext
    * 
    * <p>Only takes effect when IndexWriter is first created. */
   public IndexWriterConfig setOpenMode(OpenMode openMode) {
+    if (openMode == null) {
+      throw new IllegalArgumentException("openMode must not be null");
+    }
     this.openMode = openMode;
     return this;
   }
@@ -187,13 +194,15 @@ public final class IndexWriterConfig ext
    * like NFS that do not support "delete on last close" semantics, which
    * Lucene's "point in time" search normally relies on.
    * <p>
-   * <b>NOTE:</b> the deletion policy cannot be null. If <code>null</code> is
-   * passed, the deletion policy will be set to the default.
+   * <b>NOTE:</b> the deletion policy cannot be null.
    *
    * <p>Only takes effect when IndexWriter is first created. 
    */
   public IndexWriterConfig setIndexDeletionPolicy(IndexDeletionPolicy delPolicy) {
-    this.delPolicy = delPolicy == null ? new KeepOnlyLastCommitDeletionPolicy() : delPolicy;
+    if (delPolicy == null) {
+      throw new IllegalArgumentException("indexDeletionPolicy must not be null");
+    }
+    this.delPolicy = delPolicy;
     return this;
   }
 
@@ -220,12 +229,14 @@ public final class IndexWriterConfig ext
   /**
    * Expert: set the {@link Similarity} implementation used by this IndexWriter.
    * <p>
-   * <b>NOTE:</b> the similarity cannot be null. If <code>null</code> is passed,
-   * the similarity will be set to the default implementation (unspecified).
+   * <b>NOTE:</b> the similarity cannot be null.
    *
    * <p>Only takes effect when IndexWriter is first created. */
   public IndexWriterConfig setSimilarity(Similarity similarity) {
-    this.similarity = similarity == null ? IndexSearcher.getDefaultSimilarity() : similarity;
+    if (similarity == null) {
+      throw new IllegalArgumentException("similarity must not be null");
+    }
+    this.similarity = similarity;
     return this;
   }
 
@@ -238,12 +249,14 @@ public final class IndexWriterConfig ext
    * Expert: sets the merge scheduler used by this writer. The default is
    * {@link ConcurrentMergeScheduler}.
    * <p>
-   * <b>NOTE:</b> the merge scheduler cannot be null. If <code>null</code> is
-   * passed, the merge scheduler will be set to the default.
+   * <b>NOTE:</b> the merge scheduler cannot be null.
    *
    * <p>Only takes effect when IndexWriter is first created. */
   public IndexWriterConfig setMergeScheduler(MergeScheduler mergeScheduler) {
-    this.mergeScheduler = mergeScheduler == null ? new ConcurrentMergeScheduler() : mergeScheduler;
+    if (mergeScheduler == null) {
+      throw new IllegalArgumentException("mergeScheduler must not be null");
+    }
+    this.mergeScheduler = mergeScheduler;
     return this;
   }
 
@@ -272,12 +285,14 @@ public final class IndexWriterConfig ext
    * Expert: {@link MergePolicy} is invoked whenever there are changes to the
    * segments in the index. Its role is to select which merges to do, if any,
    * and return a {@link MergePolicy.MergeSpecification} describing the merges.
-   * It also selects merges to do for forceMerge. (The default is
-   * {@link LogByteSizeMergePolicy}.
+   * It also selects merges to do for forceMerge.
    *
    * <p>Only takes effect when IndexWriter is first created. */
   public IndexWriterConfig setMergePolicy(MergePolicy mergePolicy) {
-    this.mergePolicy = mergePolicy == null ? new LogByteSizeMergePolicy() : mergePolicy;
+    if (mergePolicy == null) {
+      throw new IllegalArgumentException("mergePolicy must not be null");
+    }
+    this.mergePolicy = mergePolicy;
     return this;
   }
 
@@ -289,7 +304,7 @@ public final class IndexWriterConfig ext
    */
   public IndexWriterConfig setCodec(Codec codec) {
     if (codec == null) {
-      throw new NullPointerException();
+      throw new IllegalArgumentException("codec must not be null");
     }
     this.codec = codec;
     return this;
@@ -378,7 +393,10 @@ public final class IndexWriterConfig ext
    *
    * <p>Only takes effect when IndexWriter is first created. */
   IndexWriterConfig setIndexingChain(IndexingChain indexingChain) {
-    this.indexingChain = indexingChain == null ? DocumentsWriterPerThread.defaultIndexingChain : indexingChain;
+    if (indexingChain == null) {
+      throw new IllegalArgumentException("indexingChain must not be null");
+    }
+    this.indexingChain = indexingChain;
     return this;
   }
 
@@ -396,6 +414,9 @@ public final class IndexWriterConfig ext
    * @see #setRAMBufferSizeMB(double)
    */
   IndexWriterConfig setFlushPolicy(FlushPolicy flushPolicy) {
+    if (flushPolicy == null) {
+      throw new IllegalArgumentException("flushPolicy must not be null");
+    }
     this.flushPolicy = flushPolicy;
     return this;
   }
@@ -484,7 +505,10 @@ public final class IndexWriterConfig ext
   
   /** Convenience method that uses {@link PrintStreamInfoStream} */
   public IndexWriterConfig setInfoStream(PrintStream printStream) {
-    return setInfoStream(printStream == null ? InfoStream.NO_OUTPUT : new PrintStreamInfoStream(printStream));
+    if (printStream == null) {
+      throw new IllegalArgumentException("printStream must not be null");
+    }
+    return setInfoStream(new PrintStreamInfoStream(printStream));
   }
   
   @Override

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java Thu May 30 07:53:18 2013
@@ -26,7 +26,7 @@ import java.util.List;
  * the default deletion policy.
  */
 
-public final class KeepOnlyLastCommitDeletionPolicy implements IndexDeletionPolicy {
+public final class KeepOnlyLastCommitDeletionPolicy extends IndexDeletionPolicy {
 
   /** Sole constructor. */
   public KeepOnlyLastCommitDeletionPolicy() {

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java Thu May 30 07:53:18 2013
@@ -545,13 +545,7 @@ public abstract class LogMergePolicy ext
     // Sorts largest to smallest
     @Override
     public int compareTo(SegmentInfoAndLevel other) {
-      if (level < other.level) {
-        return 1;
-      } else if (level > other.level) {
-        return -1;
-      } else {
-        return 0;
-      }
+      return Float.compare(other.level, level);
     }
   }