You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2013/05/13 15:24:02 UTC

svn commit: r1481840 [2/4] - in /lucene/dev/branches/lucene4258: ./ dev-tools/ dev-tools/idea/.idea/libraries/ dev-tools/maven/ dev-tools/maven/lucene/replicator/ dev-tools/scripts/ lucene/ lucene/analysis/ lucene/analysis/common/ lucene/analysis/commo...

Modified: lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/DocInverterPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/DocInverterPerField.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/DocInverterPerField.java (original)
+++ lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/DocInverterPerField.java Mon May 13 13:24:00 2013
@@ -120,10 +120,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) {
@@ -146,11 +146,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/lucene4258/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java Mon May 13 13:24:00 2013
@@ -421,7 +421,7 @@ final class DocumentsWriter {
         // create new fields update, which should effect previous docs in the
         // current segment
         FieldsUpdate fieldsUpdate = new FieldsUpdate(term, operation, fields, 
-            analyzer, numDocsInRAM.get() - 1);
+            analyzer, numDocsInRAM.get() - 1, System.currentTimeMillis());
         // invert the given fields and store in RAMDirectory
         dwpt.invertFieldsUpdate(fieldsUpdate, globalFieldNumberMap);
         dwpt.updateFields(term, fieldsUpdate);
@@ -480,6 +480,9 @@ final class DocumentsWriter {
         // abortable so that IW.close(false) is able to stop it
         TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(directory);
         
+        updates.startWriting(infoPerCommit.getNextUpdateGen(),
+            infoPerCommit.info.getDocCount());
+
         final List<AtomicReader> mergeReaders = new ArrayList<AtomicReader>();
         AtomicReader reader;
         while ((reader = updates.nextReader()) != null) { // add new indexes
@@ -489,10 +492,6 @@ final class DocumentsWriter {
         SegmentMerger merger = new SegmentMerger(mergeReaders, info, infoStream, trackingDir,
             interval, MergeState.CheckAbort.NONE, globalFieldNumberMap, context);
         
-        updates.startWriting(infoPerCommit.getNextUpdateGen(),
-            infoPerCommit.info.getDocCount(), indexWriter.getConfig()
-                .getReaderTermsIndexDivisor());
-
         Set<String> generationReplacementFilenames = null;
         boolean success = false;
         try {

Modified: lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Mon May 13 13:24:00 2013
@@ -132,13 +132,11 @@ class DocumentsWriterPerThread {
     final FieldInfos fieldInfos;
     final FrozenBufferedDeletes segmentDeletes;
     final MutableBits liveDocs;
-    final UpdatedSegmentData liveUpdates;
     final int delCount;
 
     private FlushedSegment(SegmentInfoPerCommit segmentInfo, FieldInfos fieldInfos,
                            BufferedDeletes segmentDeletes, MutableBits liveDocs, 
-                           int delCount, BufferedUpdates segmentUpdates,
-                           UpdatedSegmentData liveUpdates) {
+                           int delCount, BufferedUpdates segmentUpdates) {
       this.segmentInfo = segmentInfo;
       this.fieldInfos = fieldInfos;
       if ((segmentDeletes != null && segmentDeletes.any())
@@ -149,7 +147,6 @@ class DocumentsWriterPerThread {
         this.segmentDeletes = null;
       }
       this.liveDocs = liveDocs;
-      this.liveUpdates = liveUpdates;
       this.delCount = delCount;
     }
   }
@@ -618,9 +615,9 @@ class DocumentsWriterPerThread {
 
       assert segmentInfo != null;
 
-      FlushedSegment fs = new FlushedSegment(segmentInfoPerCommit, flushState.fieldInfos,
-                                             segmentDeletes, flushState.liveDocs, flushState.delCountOnFlush, 
-                                             pendingUpdates, flushState.liveUpdates);
+      FlushedSegment fs = new FlushedSegment(segmentInfoPerCommit,
+          flushState.fieldInfos, segmentDeletes, flushState.liveDocs,
+          flushState.delCountOnFlush, pendingUpdates);
       sealFlushedSegment(fs);
       doAfterFlush();
       success = true;

Modified: lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/FieldsUpdate.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/FieldsUpdate.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/FieldsUpdate.java (original)
+++ lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/FieldsUpdate.java Mon May 13 13:24:00 2013
@@ -46,7 +46,8 @@ public class FieldsUpdate implements Com
   final Operation operation;
   final Set<String> replacedFields;
   final Analyzer analyzer;
-  final int docIDUpto;
+  final int docIdUpto;
+  final long timeStamp;
 
   IndexDocument fields;
   Directory directory;
@@ -67,7 +68,7 @@ public class FieldsUpdate implements Com
    *          Document ID of the last document added before this field update
    */
   public FieldsUpdate(Term term, Operation operation, IndexDocument fields,
-      Analyzer analyzer, int docIDUpto) {
+      Analyzer analyzer, int docIDUpto, long timeStamp) {
     this.term = term;
     this.fields = fields;
     this.operation = operation;
@@ -83,7 +84,8 @@ public class FieldsUpdate implements Com
       }
     }
     this.analyzer = analyzer;
-    this.docIDUpto = docIDUpto;
+    this.docIdUpto = docIDUpto;
+    this.timeStamp = timeStamp;
   }
   
   /**
@@ -97,7 +99,8 @@ public class FieldsUpdate implements Com
     this.operation = other.operation;
     this.replacedFields = other.replacedFields;
     this.analyzer = other.analyzer;
-    this.docIDUpto = other.docIDUpto;
+    this.docIdUpto = other.docIdUpto;
+    this.timeStamp = other.timeStamp;
     this.directory = other.directory;
     this.segmentInfo = other.segmentInfo;
   }
@@ -105,7 +108,15 @@ public class FieldsUpdate implements Com
   /* Order FrieldsUpdate by increasing docIDUpto */
   @Override
   public int compareTo(FieldsUpdate other) {
-    return this.docIDUpto - other.docIDUpto;
+    int diff = this.docIdUpto - other.docIdUpto;
+    if (diff == 0) {
+      if (this.timeStamp < other.timeStamp) {
+        return -1;
+      } else if (this.timeStamp > other.timeStamp) {
+        return 1;
+      }
+    }
+    return diff;
   }
   
 }

Modified: lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java (original)
+++ lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java Mon May 13 13:24:00 2013
@@ -19,9 +19,7 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 import java.util.Comparator;
-import java.util.Iterator;
 import java.util.Map;
-import java.util.SortedSet;
 
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
@@ -367,13 +365,6 @@ final class FreqProxTermsWriterPerField 
       segDeletes = null;
     }
     
-    final Map<Term,SortedSet<FieldsUpdate>> segUpdates;
-    if (state.segUpdates != null && state.segUpdates.terms.size() > 0) {
-      segUpdates = state.segUpdates.terms;
-    } else {
-      segUpdates = null;
-    }
-    
     final int[] termIDs = termsHashPerField.sortPostings(termComp);
     final int numTerms = termsHashPerField.bytesHash.size();
     final BytesRef text = new BytesRef();
@@ -406,8 +397,6 @@ final class FreqProxTermsWriterPerField 
 
       final PostingsConsumer postingsConsumer = termsConsumer.startTerm(text);
 
-      Term term = new Term(fieldName, text);
-      
       final int delDocLimit;
       if (segDeletes != null) {
         protoTerm.bytes = text;
@@ -421,19 +410,6 @@ final class FreqProxTermsWriterPerField 
         delDocLimit = 0;
       }
 
-      final SortedSet<FieldsUpdate> termUpdates;
-      Iterator<FieldsUpdate> updatesIterator = null;
-      FieldsUpdate nextUpdate = null;
-      if (segUpdates != null) {
-        termUpdates = segUpdates.get(term);
-        if (termUpdates != null && !termUpdates.isEmpty()) {
-          updatesIterator = termUpdates.iterator();
-          nextUpdate = updatesIterator.next();
-        }
-      } else {
-        termUpdates = null;
-      }
-
       // Now termStates has numToMerge FieldMergeStates
       // which all share the same term.  Now we must
       // interleave the docID streams.
@@ -506,23 +482,6 @@ final class FreqProxTermsWriterPerField 
           }
         }
 
-        // make sure we update the relevant documents according to the doc ID
-        // in which the updates arrived
-        while (nextUpdate != null && docID > nextUpdate.docIDUpto) {
-          if (updatesIterator.hasNext()) {
-            nextUpdate = updatesIterator.next();
-          } else {
-            nextUpdate = null;
-          }
-        }
-        
-        if (nextUpdate != null) {
-            if (state.liveUpdates == null) {
-              state.liveUpdates = new UpdatedSegmentData();
-            }
-            state.liveUpdates.addUpdate(docID, nextUpdate, true);
-        }
-
         totTF += termFreq;
         
         // Carefully copy over the prox + payload info,

Modified: lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java (original)
+++ lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java Mon May 13 13:24:00 2013
@@ -60,23 +60,36 @@ class FrozenBufferedDeletes {
 
   public FrozenBufferedDeletes(BufferedDeletes deletes, BufferedUpdates updates, boolean isSegmentPrivate) {
     this.isSegmentPrivate = isSegmentPrivate;
-    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.timSort(termsArray);
-    PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder();
-    for (Term term : termsArray) {
-      builder.add(term);
-    }
-    terms = builder.finish();
-    
-    queries = new Query[deletes.queries.size()];
-    queryLimits = new int[deletes.queries.size()];
-    int upto = 0;
-    for(Map.Entry<Query,Integer> ent : deletes.queries.entrySet()) {
-      queries[upto] = ent.getKey();
-      queryLimits[upto] = ent.getValue();
-      upto++;
+    int localBytesUsed = 0;
+    if (deletes != null) {
+      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.timSort(termsArray);
+      PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder();
+      for (Term term : termsArray) {
+        builder.add(term);
+      }
+      terms = builder.finish();
+      localBytesUsed += (int) terms.getSizeInBytes();
+      
+      queries = new Query[deletes.queries.size()];
+      queryLimits = new int[deletes.queries.size()];
+      int upto = 0;
+      for (Map.Entry<Query,Integer> ent : deletes.queries.entrySet()) {
+        queries[upto] = ent.getKey();
+        queryLimits[upto] = ent.getValue();
+        upto++;
+      }
+      
+      localBytesUsed += queries.length * BYTES_PER_DEL_QUERY;
+      numTermDeletes = deletes.numTermDeletes.get();
+    } else { 
+      terms = null;
+      numTermDeletes = 0;
+      queries = null;
+      queryLimits = null;
     }
     
     // freeze updates
@@ -87,10 +100,10 @@ class FrozenBufferedDeletes {
       for (SortedSet<FieldsUpdate> list : updates.terms.values()) {
         allUpdates.addAll(list);
       }
+      localBytesUsed += 100;
     }
     
-    bytesUsed = (int) terms.getSizeInBytes() + queries.length * BYTES_PER_DEL_QUERY + 100 /* updates */;
-    numTermDeletes = deletes.numTermDeletes.get();
+    bytesUsed = localBytesUsed;
   }
   
   public void setDelGen(long gen) {

Modified: lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/IndexFileNames.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/IndexFileNames.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/IndexFileNames.java (original)
+++ lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/IndexFileNames.java Mon May 13 13:24:00 2013
@@ -238,11 +238,15 @@ 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]+(_.*)?\\..*");
 
+  /** Returns true if the file denotes an updated segment. */
   public static boolean isUpdatedSegmentFile(String file) {
     return file.startsWith("__");
   }
+  
 }

Modified: lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Mon May 13 13:24:00 2013
@@ -384,7 +384,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);
@@ -452,6 +452,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:
@@ -575,8 +585,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;
+    }
   }
   
   /**
@@ -699,8 +722,7 @@ public class IndexWriter implements Clos
         
         // Record that we have a change (zero out all
         // segments) pending:
-        changeCount++;
-        segmentInfos.changed();
+        changed();
       } else {
         segmentInfos.read(directory);
         
@@ -716,8 +738,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() + "\"");
@@ -746,8 +767,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")) {
@@ -1118,6 +1138,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;
@@ -1524,7 +1547,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;
@@ -2411,9 +2434,14 @@ public class IndexWriter implements Clos
    * referenced exist (correctly) in the 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);
   }
   
   void writeSegmentUpdates(SegmentInfoPerCommit info,
@@ -4513,7 +4541,6 @@ public class IndexWriter implements Clos
               + (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/lucene4258/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java Mon May 13 13:24:00 2013
@@ -242,6 +242,39 @@ public final class SegmentInfos implemen
     }
   }
 
+  /**
+   * A utility for writing the {@link IndexFileNames#SEGMENTS_GEN} file to a
+   * {@link Directory}.
+   * 
+   * <p>
+   * <b>NOTE:</b> this is an internal utility which is kept public so that it's
+   * accessible by code from other packages. You should avoid calling this
+   * method unless you're absolutely sure what you're doing!
+   * 
+   * @lucene.internal
+   */
+  public static void writeSegmentsGen(Directory dir, long generation) {
+    try {
+      IndexOutput genOutput = dir.createOutput(IndexFileNames.SEGMENTS_GEN, IOContext.READONCE);
+      try {
+        genOutput.writeInt(FORMAT_SEGMENTS_GEN_CURRENT);
+        genOutput.writeLong(generation);
+        genOutput.writeLong(generation);
+      } finally {
+        genOutput.close();
+        dir.sync(Collections.singleton(IndexFileNames.SEGMENTS_GEN));
+      }
+    } catch (Throwable t) {
+      // It's OK if we fail to write this file since it's
+      // used only as one of the retry fallbacks.
+      try {
+        dir.deleteFile(IndexFileNames.SEGMENTS_GEN);
+      } catch (Throwable t2) {
+        // Ignore; this file is only used in a retry
+        // fallback on init.
+      }
+    }
+  }
 
   /**
    * Get the next segments_N filename that will be written.
@@ -850,27 +883,7 @@ public final class SegmentInfos implemen
     }
 
     lastGeneration = generation;
-
-    try {
-      IndexOutput genOutput = dir.createOutput(IndexFileNames.SEGMENTS_GEN, IOContext.READONCE);
-      try {
-        genOutput.writeInt(FORMAT_SEGMENTS_GEN_CURRENT);
-        genOutput.writeLong(generation);
-        genOutput.writeLong(generation);
-      } finally {
-        genOutput.close();
-        dir.sync(Collections.singleton(IndexFileNames.SEGMENTS_GEN));
-      }
-    } catch (Throwable t) {
-      // It's OK if we fail to write this file since it's
-      // used only as one of the retry fallbacks.
-      try {
-        dir.deleteFile(IndexFileNames.SEGMENTS_GEN);
-      } catch (Throwable t2) {
-        // Ignore; this file is only used in a retry
-        // fallback on init.
-      }
-    }
+    writeSegmentsGen(dir, generation);
   }
 
   /** Writes & syncs to the Directory dir, taking care to

Modified: lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java (original)
+++ lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java Mon May 13 13:24:00 2013
@@ -67,9 +67,6 @@ public class SegmentWriteState {
    *  only set if there is one or more deleted documents. */
   public MutableBits liveDocs;
 
-  // Lazily created:
-  public UpdatedSegmentData liveUpdates;
-  
   /** Unique suffix for any postings files written for this
    *  segment.  {@link PerFieldPostingsFormat} sets this for
    *  each of the postings formats it wraps.  If you create

Modified: lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java (original)
+++ lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java Mon May 13 13:24:00 2013
@@ -89,6 +89,10 @@ final class StandardDirectoryReader exte
       IOException prior = null;
       boolean success = false;
       try {
+        // NOTE: important that we use infos not
+        // segmentInfos here, so that we are passing the
+        // actual instance of SegmentInfoPerCommit in
+        // IndexWriter's segmentInfos:
         final SegmentInfoPerCommit info = infos.info(i);
         assert info.info.dir == dir;
         final ReadersAndLiveDocs rld = writer.readerPool.get(info, true);
@@ -118,9 +122,10 @@ final class StandardDirectoryReader exte
       writer, segmentInfos, writer.getConfig().getReaderTermsIndexDivisor(), applyAllDeletes);
   }
 
-  /** This constructor is only used for {@link #doOpenIfChanged(SegmentInfos, IndexWriter)} */
-  private static DirectoryReader open(Directory directory, IndexWriter writer, SegmentInfos infos, List<? extends AtomicReader> oldReaders,
+  /** This constructor is only used for {@link #doOpenIfChanged(SegmentInfos)} */
+  private static DirectoryReader open(Directory directory, SegmentInfos infos, List<? extends AtomicReader> oldReaders,
     int termInfosIndexDivisor) throws IOException {
+
     // we put the old SegmentReaders in a map, that allows us
     // to lookup a reader using its segment name
     final Map<String,Integer> segmentReaders = new HashMap<String,Integer>();
@@ -208,7 +213,7 @@ final class StandardDirectoryReader exte
         }
       }
     }    
-    return new StandardDirectoryReader(directory, newReaders, writer, infos, termInfosIndexDivisor, false);
+    return new StandardDirectoryReader(directory, newReaders, null, infos, termInfosIndexDivisor, false);
   }
 
   @Override
@@ -233,7 +238,7 @@ final class StandardDirectoryReader exte
 
   @Override
   protected DirectoryReader doOpenIfChanged() throws IOException {
-    return doOpenIfChanged(null);
+    return doOpenIfChanged((IndexCommit) null);
   }
 
   @Override
@@ -303,13 +308,13 @@ final class StandardDirectoryReader exte
       protected Object doBody(String segmentFileName) throws IOException {
         final SegmentInfos infos = new SegmentInfos();
         infos.read(directory, segmentFileName);
-        return doOpenIfChanged(infos, null);
+        return doOpenIfChanged(infos);
       }
     }.run(commit);
   }
 
-  DirectoryReader doOpenIfChanged(SegmentInfos infos, IndexWriter writer) throws IOException {
-    return StandardDirectoryReader.open(directory, writer, infos, getSequentialSubReaders(), termInfosIndexDivisor);
+  DirectoryReader doOpenIfChanged(SegmentInfos infos) throws IOException {
+    return StandardDirectoryReader.open(directory, infos, getSequentialSubReaders(), termInfosIndexDivisor);
   }
 
   @Override

Modified: lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/UpdatedSegmentData.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/UpdatedSegmentData.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/UpdatedSegmentData.java (original)
+++ lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/UpdatedSegmentData.java Mon May 13 13:24:00 2013
@@ -1,14 +1,21 @@
 package org.apache.lucene.index;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.SortedSet;
 import java.util.TreeMap;
 
 import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.index.FieldsUpdate.Operation;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.Bits;
@@ -36,44 +43,96 @@ import org.apache.lucene.util.Bits;
 class UpdatedSegmentData {
   
   static final FieldInfos EMPTY_FIELD_INFOS = new FieldInfos(new FieldInfo[0]);
-
+  
   /** Updates mapped by doc ID, for each do sorted list of updates. */
-  private TreeMap<Integer,PriorityQueue<FieldsUpdate>> updatesMap;
+  private TreeMap<Integer,TreeMap<FieldsUpdate, Set<String>>> docIdToUpdatesMap;
+  private HashMap<FieldsUpdate, List<Integer>> updatesToDocIdMap;
+  private LinkedHashMap<FieldsUpdate,UpdateAtomicReader> allApplied;
   
-  /** */
   private long generation;
+  private boolean exactSegment;
   
-  private Map<String,FieldGenerationReplacements> fieldGenerationReplacments = new HashMap<String,FieldGenerationReplacements>();
+  private Map<String,FieldGenerationReplacements> fieldGenerationReplacments;
   
-  private Iterator<Entry<Integer,PriorityQueue<FieldsUpdate>>> updatesIterator;
+  private Iterator<Entry<Integer,TreeMap<FieldsUpdate,Set<String>>>> updatesIterator;
   private int currDocID;
   private int nextDocID;
   private int numDocs;
-  private PriorityQueue<FieldsUpdate> nextUpdate;
+  private TreeMap<FieldsUpdate,Set<String>> nextUpdate;
   private Analyzer analyzer;
   
-  private int termsIndexDivisor;
-  
-  UpdatedSegmentData() {
-    updatesMap = new TreeMap<Integer,PriorityQueue<FieldsUpdate>>();
+  UpdatedSegmentData(SegmentReader reader,
+      SortedSet<FieldsUpdate> packetUpdates, boolean exactSegment)
+      throws IOException {
+    docIdToUpdatesMap = new TreeMap<>();
+    updatesToDocIdMap = new HashMap<>();
+    this.exactSegment = exactSegment;
+    
+    allApplied = new LinkedHashMap<>();
+    
+    for (FieldsUpdate update : packetUpdates) {
+      // add updates according to the base reader
+      DocsEnum docsEnum = reader.termDocsEnum(update.term);
+      if (docsEnum != null) {
+        int docId;
+        while ((docId = docsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+          addUpdate(docId, update);
+        }
+      }
+      
+      // try applying on previous updates in this packet
+      for (Entry<FieldsUpdate,UpdateAtomicReader> applied : allApplied
+          .entrySet()) {
+        if (applied.getValue().hasTerm(update.term)) {
+          List<Integer> list = updatesToDocIdMap.get(applied.getKey());
+          if (list != null) {
+            for (Integer docId : list) {
+              Set<String> ignoredFields = docIdToUpdatesMap.get(docId).get(
+                  applied.getKey());
+              if (ignoredFields == null
+                  || !ignoredFields.contains(update.term.field())) {
+                addUpdate(docId, update);
+              }
+            }
+          }
+        }
+      }
+      
+      allApplied.put(update, new UpdateAtomicReader(update.directory,
+          update.segmentInfo, IOContext.DEFAULT));
+    }
+    
   }
   
-  void addUpdate(int docId, FieldsUpdate fieldsUpdate, boolean checkDocId) {
-    if (checkDocId && docId > fieldsUpdate.docIDUpto) {
+  private void addUpdate(int docId, FieldsUpdate fieldsUpdate) {
+    if (exactSegment && docId > fieldsUpdate.docIdUpto) {
       return;
     }
-    PriorityQueue<FieldsUpdate> prevUpdates = updatesMap.get(docId);
+    TreeMap<FieldsUpdate,Set<String>> prevUpdates = docIdToUpdatesMap.get(docId);
     if (prevUpdates == null) {
-      prevUpdates = new PriorityQueue<FieldsUpdate>();
-      updatesMap.put(docId, prevUpdates);
-    } else {
-      System.out.println();
+      prevUpdates = new TreeMap<>();
+      docIdToUpdatesMap.put(docId, prevUpdates);
+    } else if (fieldsUpdate.operation == Operation.REPLACE_FIELDS) {
+      // set ignored fields in previous updates
+      for (Entry<FieldsUpdate,Set<String>> addIgnore : prevUpdates.entrySet()) {
+        if (addIgnore.getValue() == null) {
+          prevUpdates.put(addIgnore.getKey(), new HashSet<>(fieldsUpdate.replacedFields));
+        } else {
+          addIgnore.getValue().addAll(fieldsUpdate.replacedFields);
+        }
+      }
+    }
+    prevUpdates.put(fieldsUpdate, null);
+    List<Integer> prevDocIds = updatesToDocIdMap.get(fieldsUpdate);
+    if (prevDocIds == null) {
+      prevDocIds = new ArrayList<Integer>();
+      updatesToDocIdMap.put(fieldsUpdate, prevDocIds);
     }
-    prevUpdates.add(fieldsUpdate);
+    prevDocIds.add(docId);
   }
   
   boolean hasUpdates() {
-    return !updatesMap.isEmpty();
+    return !docIdToUpdatesMap.isEmpty();
   }
   
   /**
@@ -83,16 +142,13 @@ class UpdatedSegmentData {
    *          The updates generation.
    * @param numDocs
    *          number of documents in the base segment
-   * @param termsIndexDivisor
-   *          Terms index divisor to use in temporary segments
    */
-  void startWriting(long generation, int numDocs, int termsIndexDivisor) {
+  void startWriting(long generation, int numDocs) {
     this.generation = generation;
     this.numDocs = numDocs;
-    this.termsIndexDivisor = termsIndexDivisor;
-    updatesIterator = updatesMap.entrySet().iterator();
+    
+    updatesIterator = docIdToUpdatesMap.entrySet().iterator();
     currDocID = 0;
-    fieldGenerationReplacments.clear();
     // fetch the first actual updates document if exists
     nextDocUpdate();
   }
@@ -102,8 +158,7 @@ class UpdatedSegmentData {
    */
   private void nextDocUpdate() {
     if (updatesIterator.hasNext()) {
-      Entry<Integer,PriorityQueue<FieldsUpdate>> docUpdates = updatesIterator
-          .next();
+      Entry<Integer,TreeMap<FieldsUpdate,Set<String>>> docUpdates = updatesIterator.next();
       nextDocID = docUpdates.getKey();
       nextUpdate = docUpdates.getValue();
     } else {
@@ -128,9 +183,9 @@ class UpdatedSegmentData {
       currDocID = nextDocID;
     } else if (currDocID < numDocs) {
       // get the an actual updates reader...
-      FieldsUpdate update = nextUpdate.poll();
-      toReturn = new UpdateAtomicReader(update.directory, update.segmentInfo,
-          IOContext.DEFAULT);
+      FieldsUpdate update = nextUpdate.firstEntry().getKey();
+      Set<String> ignore = nextUpdate.remove(update);
+      toReturn = allApplied.get(update);
       
       // ... and if done for this document remove from updates map
       if (nextUpdate.isEmpty()) {
@@ -139,6 +194,9 @@ class UpdatedSegmentData {
       
       // add generation replacements if exist
       if (update.replacedFields != null) {
+        if (fieldGenerationReplacments == null) {
+          fieldGenerationReplacments = new HashMap<String,FieldGenerationReplacements>();
+        }
         for (String fieldName : update.replacedFields) {
           FieldGenerationReplacements fieldReplacement = fieldGenerationReplacments
               .get(fieldName);
@@ -158,9 +216,9 @@ class UpdatedSegmentData {
   }
   
   boolean isEmpty() {
-    return updatesMap.isEmpty();
+    return docIdToUpdatesMap.isEmpty();
   }
-
+  
   private class UpdateAtomicReader extends AtomicReader {
     
     final private SegmentCoreReaders core;
@@ -180,8 +238,7 @@ class UpdatedSegmentData {
      */
     UpdateAtomicReader(Directory fieldsDir, SegmentInfo segmentInfo,
         IOContext context) throws IOException {
-      core = new SegmentCoreReaders(null, segmentInfo, -1, context,
-          termsIndexDivisor);
+      core = new SegmentCoreReaders(null, segmentInfo, -1, context, -1);
       numDocs = 1;
     }
     
@@ -193,6 +250,17 @@ class UpdatedSegmentData {
       this.numDocs = numDocs;
     }
     
+    boolean hasTerm(Term term) throws IOException {
+      if (core == null) {
+        return false;
+      }
+      DocsEnum termDocsEnum = termDocsEnum(term);
+      if (termDocsEnum == null) {
+        return false;
+      }
+      return termDocsEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS;
+    }
+
     @Override
     public Fields fields() throws IOException {
       if (core == null) {
@@ -247,8 +315,13 @@ class UpdatedSegmentData {
     }
     
     @Override
-    protected void doClose() throws IOException {}
-
+    protected void doClose() throws IOException {
+      if (core == null) {
+        return;
+      }
+      core.decRef();
+    }
+    
     @Override
     public NumericDocValues getNumericDocValues(String field)
         throws IOException {
@@ -257,7 +330,7 @@ class UpdatedSegmentData {
       }
       return core.getNumericDocValues(field);
     }
-
+    
     @Override
     public BinaryDocValues getBinaryDocValues(String field) throws IOException {
       if (core == null) {
@@ -265,7 +338,7 @@ class UpdatedSegmentData {
       }
       return core.getBinaryDocValues(field);
     }
-
+    
     @Override
     public SortedDocValues getSortedDocValues(String field) throws IOException {
       if (core == null) {
@@ -273,7 +346,7 @@ class UpdatedSegmentData {
       }
       return core.getSortedDocValues(field);
     }
-
+    
     @Override
     public SortedSetDocValues getSortedSetDocValues(String field)
         throws IOException {
@@ -282,7 +355,7 @@ class UpdatedSegmentData {
       }
       return core.getSortedSetDocValues(field);
     }
-
+    
     @Override
     public NumericDocValues getNormValues(String field) throws IOException {
       if (core == null) {
@@ -290,6 +363,6 @@ class UpdatedSegmentData {
       }
       return core.getNormValues(field);
     }
-    
   }
+  
 }

Modified: lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/search/LiveFieldValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/search/LiveFieldValues.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/search/LiveFieldValues.java (original)
+++ lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/search/LiveFieldValues.java Mon May 13 13:24:00 2013
@@ -33,14 +33,14 @@ import java.util.concurrent.ConcurrentHa
  *  the same time by two threads, because in this case you
  *  cannot in general know which thread "won". */
 
-public abstract class LiveFieldValues<T> implements ReferenceManager.RefreshListener, Closeable {
+public abstract class LiveFieldValues<S,T> implements ReferenceManager.RefreshListener, Closeable {
 
   private volatile Map<String,T> current = new ConcurrentHashMap<String,T>();
   private volatile Map<String,T> old = new ConcurrentHashMap<String,T>();
-  private final ReferenceManager<IndexSearcher> mgr;
+  private final ReferenceManager<S> mgr;
   private final T missingValue;
 
-  public LiveFieldValues(ReferenceManager<IndexSearcher> mgr, T missingValue) {
+  public LiveFieldValues(ReferenceManager<S> mgr, T missingValue) {
     this.missingValue = missingValue;
     this.mgr = mgr;
     mgr.addListener(this);
@@ -114,7 +114,7 @@ public abstract class LiveFieldValues<T>
         // It either does not exist in the index, or, it was
         // already flushed & NRT reader was opened on the
         // segment, so fallback to current searcher:
-        IndexSearcher s = mgr.acquire();
+        S s = mgr.acquire();
         try {
           return lookupFromSearcher(s, id);
         } finally {
@@ -128,6 +128,6 @@ public abstract class LiveFieldValues<T>
    *  in an NRT IndexSearcher.  You must implement this to
    *  go look up the value (eg, via doc values, field cache,
    *  stored fields, etc.). */
-  protected abstract T lookupFromSearcher(IndexSearcher s, String id) throws IOException;
+  protected abstract T lookupFromSearcher(S s, String id) throws IOException;
 }
 

Modified: lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/index/TestFieldReplacements.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/index/TestFieldReplacements.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/index/TestFieldReplacements.java (original)
+++ lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/index/TestFieldReplacements.java Mon May 13 13:24:00 2013
@@ -132,17 +132,12 @@ public class TestFieldReplacements exten
   }
   
   private static void addDocuments(Directory directory, Random localRandom,
-      int maxDocs, boolean randomConfig) throws IOException {
+      int maxDocs) throws IOException {
     init(localRandom);
     HashSet<Term> usedTerms = new HashSet<Term>();
     
-    final IndexWriterConfig config;
-    if (randomConfig) {
-      config = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
-    } else {
-      config = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
-    }
-    System.out.println(config.getMergePolicy());
+    final IndexWriterConfig config = newIndexWriterConfig(TEST_VERSION_CURRENT,
+        new MockAnalyzer(random()));
     config.setCodec(new SimpleTextCodec());
     IndexWriter writer = new IndexWriter(directory, config);
     
@@ -273,13 +268,13 @@ public class TestFieldReplacements exten
   }
   
   public void testRandomIndexGeneration() throws IOException {
-    addDocuments(dir, random(), Integer.MAX_VALUE, true);
+    addDocuments(dir, random(), Integer.MAX_VALUE);
     DirectoryReader directoryReader = DirectoryReader.open(dir);
     directoryReader.close();
   }
   
   public void testAddIndexes() throws IOException {
-    addDocuments(dir, random(), Integer.MAX_VALUE, true);
+    addDocuments(dir, random(), Integer.MAX_VALUE);
     RAMDirectory addedDir = new RAMDirectory();
     IndexWriter addedIndexWriter = new IndexWriter(addedDir,
         newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())));
@@ -300,13 +295,10 @@ public class TestFieldReplacements exten
   
   public void testIndexEquality() throws IOException {
     // create index through updates
-    addDocuments(dir, new Random(3), Integer.MAX_VALUE, true);
+    addDocuments(dir, new Random(3), Integer.MAX_VALUE);
     
     DirectoryReader updatesReader = DirectoryReader.open(dir);
     IndexData updatesIndexData = new IndexData(updatesReader);
-    System.out.println("Updates index data");
-    System.out.println(updatesIndexData.toString(false));
-    System.out.println();
     updatesReader.close();
     
     // create the same index directly
@@ -598,9 +590,6 @@ public class TestFieldReplacements exten
     DirectoryReader directReader = DirectoryReader.open(directDir);
     
     IndexData directIndexData = new IndexData(directReader);
-    System.out.println("Direct index data");
-    System.out.println(directIndexData.toString(false));
-    System.out.println();
     directReader.close();
     directDir.close();
     
@@ -783,6 +772,81 @@ public class TestFieldReplacements exten
     
   }
   
+  public void testReplaceLayers() throws IOException {
+    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(
+        TEST_VERSION_CURRENT, new MockAnalyzer(random())));
+    
+    FieldType fieldType = new FieldType();
+    fieldType.setIndexed(true);
+    fieldType.setTokenized(false);
+    fieldType.setOmitNorms(true);
+    fieldType.setStored(true);
+    
+    Document doc0 = new Document();
+    doc0.add(new StoredField("f1", "a", fieldType));
+    writer.addDocument(doc0);
+
+    // add f2:b
+    Document fields1 = new Document();
+    fields1.add(new StoredField("f2", "b", fieldType));
+    writer.updateFields(Operation.ADD_FIELDS, new Term("f1", "a"), fields1);
+    
+    // remove f2:b and add f2:c
+    Document fields2 = new Document();
+    fields2.add(new StoredField("f2", "c", fieldType));
+    writer.updateFields(Operation.REPLACE_FIELDS, new Term("f2", "b"), fields2);
+    
+    // do nothing since f2:b was removed
+    Document fields3 = new Document();
+    fields3.add(new StoredField("f2", "d", fieldType));
+    writer.updateFields(Operation.ADD_FIELDS, new Term("f2", "b"), fields3);
+    
+    writer.close();
+    
+    DirectoryReader directoryReader = DirectoryReader.open(dir);
+    final AtomicReader atomicReader = directoryReader.leaves().get(0).reader();
+    printField(atomicReader, "f1");
+    
+    // check indexed fields
+    final DocsAndPositionsEnum termPositionsA = atomicReader
+        .termPositionsEnum(new Term("f1", "a"));
+    assertNotNull("no positions for term", termPositionsA);
+    assertEquals("wrong doc id", 0, termPositionsA.nextDoc());
+    assertEquals("wrong position", 0, termPositionsA.nextPosition());
+    assertEquals("wrong doc id", DocIdSetIterator.NO_MORE_DOCS,
+        termPositionsA.nextDoc());
+    
+    final DocsAndPositionsEnum termPositionsB = atomicReader
+        .termPositionsEnum(new Term("f2", "b"));
+    assertNotNull("no positions for term", termPositionsB);
+    assertEquals("wrong doc id", DocIdSetIterator.NO_MORE_DOCS,
+        termPositionsB.nextDoc());
+    
+    final DocsAndPositionsEnum termPositionsC = atomicReader
+        .termPositionsEnum(new Term("f2", "c"));
+    assertNotNull("no positions for term", termPositionsC);
+    assertEquals("wrong doc id", 0, termPositionsC.nextDoc());
+    assertEquals("wrong position", 100000, termPositionsC.nextPosition());
+    assertEquals("wrong doc id", DocIdSetIterator.NO_MORE_DOCS,
+        termPositionsC.nextDoc());
+    
+    final DocsAndPositionsEnum termPositionsD = atomicReader
+        .termPositionsEnum(new Term("f2", "d"));
+    assertNull("unexpected positions for term", termPositionsD);
+    
+    // check stored fields
+    final StoredDocument stored0 = atomicReader.document(0);
+    final StorableField[] f1_0 = stored0.getFields("f1");
+    assertEquals("wrong numeber of stored fields", 1, f1_0.length);
+    assertEquals("wrong field value", "a", f1_0[0].stringValue());
+    final StorableField[] f2_0 = stored0.getFields("f2");
+    assertEquals("wrong numeber of stored fields", 1, f2_0.length);
+    assertEquals("wrong field value", "c", f2_0[0].stringValue());
+    
+    directoryReader.close();
+    
+  }
+  
   private void printField(AtomicReader atomicReader, String fieldName)
       throws IOException {
     if (!VERBOSE_FIELD_REPLACEMENTS) {
@@ -807,8 +871,8 @@ public class TestFieldReplacements exten
     }
   }
   
-  public void testprintIndexes() throws IOException {
-    File outDir = new File("D:/temp/ifu/compare/scenario/a");
+  public void printIndexes() throws IOException {
+    File outDir = new File("D:/temp/ifu/compare/scenario/b");
     outDir.mkdirs();
     
     for (int i = 0; i < 42; i++) {
@@ -819,8 +883,7 @@ public class TestFieldReplacements exten
       for (String filename : directory.listAll()) {
         new File(fsDirFile, filename).delete();
       }
-      System.out.print("" + i + " ");
-      addDocuments(directory, new Random(3), i, true);
+      addDocuments(directory, new Random(3), i);
       DirectoryReader updatesReader = DirectoryReader.open(directory);
       IndexData updatesIndexData = new IndexData(updatesReader);
       updatesReader.close();

Modified: lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/search/TestLiveFieldValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/search/TestLiveFieldValues.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/search/TestLiveFieldValues.java (original)
+++ lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/search/TestLiveFieldValues.java Mon May 13 13:24:00 2013
@@ -58,7 +58,7 @@ public class TestLiveFieldValues extends
 
     final Integer missing = -1;
 
-    final LiveFieldValues<Integer> rt = new LiveFieldValues<Integer>(mgr, missing) {
+    final LiveFieldValues<IndexSearcher,Integer> rt = new LiveFieldValues<IndexSearcher,Integer>(mgr, missing) {
         @Override
         protected Integer lookupFromSearcher(IndexSearcher s, String id) throws IOException {
           TermQuery tq = new TermQuery(new Term("id", id));

Modified: lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/search/TestPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/search/TestPhraseQuery.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/search/TestPhraseQuery.java (original)
+++ lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/search/TestPhraseQuery.java Mon May 13 13:24:00 2013
@@ -38,13 +38,6 @@ import com.carrotsearch.randomizedtestin
  *
  * @see TestPositionIncrement
  */ 
-/*
- * Remove ThreadLeaks and run with (Eclipse or command line):
- * -ea -Drt.seed=AFD1E7E84B35D2B1
- * to get leaked thread errors.
- */
-// @ThreadLeaks(linger = 1000, leakedThreadsBelongToSuite = true)
-@Seed("AFD1E7E84B35D2B1")
 public class TestPhraseQuery extends LuceneTestCase {
 
   /** threshold for comparing floats */

Modified: lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/util/TestMaxFailuresRule.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/util/TestMaxFailuresRule.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/util/TestMaxFailuresRule.java (original)
+++ lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/util/TestMaxFailuresRule.java Mon May 13 13:24:00 2013
@@ -66,13 +66,10 @@ public class TestMaxFailuresRule extends
 
   @Test
   public void testMaxFailures() {
-    int maxFailures = LuceneTestCase.ignoreAfterMaxFailures.maxFailures;
-    int failuresSoFar = LuceneTestCase.ignoreAfterMaxFailures.failuresSoFar;
+    TestRuleIgnoreAfterMaxFailures newRule = new TestRuleIgnoreAfterMaxFailures(2);
+    TestRuleIgnoreAfterMaxFailures prevRule = LuceneTestCase.replaceMaxFailureRule(newRule);
     System.clearProperty(SysGlobals.SYSPROP_ITERATIONS());
     try {
-      LuceneTestCase.ignoreAfterMaxFailures.maxFailures = 2;
-      LuceneTestCase.ignoreAfterMaxFailures.failuresSoFar = 0;
-
       JUnitCore core = new JUnitCore();
       final StringBuilder results = new StringBuilder();
       core.addListener(new RunListener() {
@@ -110,8 +107,7 @@ public class TestMaxFailuresRule extends
           results.toString().matches("(S*F){2}A+"));
 
     } finally {
-      LuceneTestCase.ignoreAfterMaxFailures.maxFailures = maxFailures;
-      LuceneTestCase.ignoreAfterMaxFailures.failuresSoFar = failuresSoFar;
+      LuceneTestCase.replaceMaxFailureRule(prevRule);
     }
   }
 }

Modified: lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java (original)
+++ lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java Mon May 13 13:24:00 2013
@@ -593,7 +593,7 @@ public class TestFSTs extends LuceneTest
   // TODO: can FST be used to index all internal substrings,
   // mapping to term?
 
-  // java -cp ../build/codecs/classes/java:../test-framework/lib/randomizedtesting-runner-2.0.9.jar:../build/core/classes/test:../build/core/classes/test-framework:../build/core/classes/java:../build/test-framework/classes/java:../test-framework/lib/junit-4.10.jar org.apache.lucene.util.fst.TestFSTs /xold/tmp/allTerms3.txt out
+  // java -cp ../build/codecs/classes/java:../test-framework/lib/randomizedtesting-runner-2.0.10.jar:../build/core/classes/test:../build/core/classes/test-framework:../build/core/classes/java:../build/test-framework/classes/java:../test-framework/lib/junit-4.10.jar org.apache.lucene.util.fst.TestFSTs /xold/tmp/allTerms3.txt out
   public static void main(String[] args) throws IOException {
     int prune = 0;
     int limit = Integer.MAX_VALUE;

Modified: lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/util/junitcompat/WithNestedTests.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/util/junitcompat/WithNestedTests.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/util/junitcompat/WithNestedTests.java (original)
+++ lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/util/junitcompat/WithNestedTests.java Mon May 13 13:24:00 2013
@@ -23,11 +23,13 @@ import java.io.UnsupportedEncodingExcept
 import java.util.List;
 
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestRuleIgnoreAfterMaxFailures;
 import org.apache.lucene.util.TestRuleIgnoreTestSuites;
 import org.apache.lucene.util.TestRuleMarkFailure;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
+import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.rules.RuleChain;
 import org.junit.rules.TestRule;
@@ -66,6 +68,22 @@ public abstract class WithNestedTests {
   private ByteArrayOutputStream sysout;
   private ByteArrayOutputStream syserr;
 
+  @ClassRule
+  public static final TestRule classRules = RuleChain.outerRule(new TestRuleAdapter() {
+    private TestRuleIgnoreAfterMaxFailures prevRule;
+
+    protected void before() throws Throwable {
+      TestRuleIgnoreAfterMaxFailures newRule = new TestRuleIgnoreAfterMaxFailures(Integer.MAX_VALUE);
+      prevRule = LuceneTestCase.replaceMaxFailureRule(newRule);
+    }
+
+    protected void afterAlways(List<Throwable> errors) throws Throwable {
+      if (prevRule != null) {
+        LuceneTestCase.replaceMaxFailureRule(prevRule);
+      }
+    }
+  }); 
+
   /**
    * Restore properties after test.
    */
@@ -86,7 +104,7 @@ public abstract class WithNestedTests {
       })
       .around(marker);
   }
-      
+
   @Before
   public final void before() {
     if (suppressOutputStreams) {

Modified: lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/associations/MultiAssociationsFacetsAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/associations/MultiAssociationsFacetsAggregator.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/associations/MultiAssociationsFacetsAggregator.java (original)
+++ lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/associations/MultiAssociationsFacetsAggregator.java Mon May 13 13:24:00 2013
@@ -48,7 +48,7 @@ public class MultiAssociationsFacetsAggr
    * Creates a new {@link MultiAssociationsFacetsAggregator} over the given
    * aggregators. The mapping is used by
    * {@link #rollupValues(FacetRequest, int, int[], int[], FacetArrays)} to
-   * rollup the values of the speicfic category by the corresponding
+   * rollup the values of the specific category by the corresponding
    * {@link FacetsAggregator}. However, since each {@link FacetsAggregator}
    * handles the associations of a specific type, which could cover multiple
    * categories, the aggregation is done on the unique set of aggregators, which

Modified: lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/range/RangeAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/range/RangeAccumulator.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/range/RangeAccumulator.java (original)
+++ lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/range/RangeAccumulator.java Mon May 13 13:24:00 2013
@@ -119,7 +119,7 @@ public class RangeAccumulator extends Fa
   }
 
   @Override
-  protected boolean requiresDocScores() {
+  public boolean requiresDocScores() {
     return false;
   }
 }

Modified: lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/DrillDownQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/DrillDownQuery.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/DrillDownQuery.java (original)
+++ lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/DrillDownQuery.java Mon May 13 13:24:00 2013
@@ -21,7 +21,6 @@ import java.io.IOException;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.regex.Pattern;
 
 import org.apache.lucene.facet.params.CategoryListParams;
 import org.apache.lucene.facet.params.FacetIndexingParams;
@@ -90,7 +89,7 @@ public final class DrillDownQuery extend
   }
 
   /** Used by DrillSideways */
-  DrillDownQuery(FacetIndexingParams fip, Query baseQuery, List<Query> clauses) {
+  DrillDownQuery(FacetIndexingParams fip, Query baseQuery, List<Query> clauses, Map<String,Integer> drillDownDims) {
     this.fip = fip;
     this.query = new BooleanQuery(true);
     if (baseQuery != null) {
@@ -98,21 +97,8 @@ public final class DrillDownQuery extend
     }
     for(Query clause : clauses) {
       query.add(clause, Occur.MUST);
-      drillDownDims.put(getDim(clause), drillDownDims.size());
     }
-  }
-
-  String getDim(Query clause) {
-    assert clause instanceof ConstantScoreQuery;
-    clause = ((ConstantScoreQuery) clause).getQuery();
-    assert clause instanceof TermQuery || clause instanceof BooleanQuery;
-    String term;
-    if (clause instanceof TermQuery) {
-      term = ((TermQuery) clause).getTerm().text();
-    } else {
-      term = ((TermQuery) ((BooleanQuery) clause).getClauses()[0].getQuery()).getTerm().text();
-    }
-    return term.split(Pattern.quote(Character.toString(fip.getFacetDelimChar())), 2)[0];
+    this.drillDownDims.putAll(drillDownDims);
   }
 
   /**

Modified: lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java (original)
+++ lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java Mon May 13 13:24:00 2013
@@ -20,6 +20,7 @@ package org.apache.lucene.facet.search;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashSet;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -94,6 +95,11 @@ public class DrillSideways {
     BooleanClause[] clauses = in.getBooleanQuery().getClauses();
     Map<String,Integer> drillDownDims = in.getDims();
 
+    String[] dimsByIndex = new String[drillDownDims.size()];
+    for(Map.Entry<String,Integer> ent : drillDownDims.entrySet()) {
+      dimsByIndex[ent.getValue()] = ent.getKey();
+    }
+
     int startClause;
     if (clauses.length == drillDownDims.size()) {
       startClause = 0;
@@ -107,13 +113,15 @@ public class DrillSideways {
     // baseQuery:
     List<Query> nonFacetClauses = new ArrayList<Query>();
     List<Query> facetClauses = new ArrayList<Query>();
+    Map<String,Integer> dimToIndex = new LinkedHashMap<String,Integer>();
     for(int i=startClause;i<clauses.length;i++) {
       Query q = clauses[i].getQuery();
-      String dim = in.getDim(q);
+      String dim = dimsByIndex[i-startClause];
       if (!facetDims.contains(dim)) {
         nonFacetClauses.add(q);
       } else {
         facetClauses.add(q);
+        dimToIndex.put(dim, dimToIndex.size());
       }
     }
 
@@ -127,7 +135,7 @@ public class DrillSideways {
         newBaseQuery.add(q, BooleanClause.Occur.MUST);
       }
 
-      return new DrillDownQuery(fsp.indexingParams, newBaseQuery, facetClauses);
+      return new DrillDownQuery(fsp.indexingParams, newBaseQuery, facetClauses, dimToIndex);
     } else {
       // No change:
       return in;
@@ -157,6 +165,20 @@ public class DrillSideways {
       return new DrillSidewaysResult(c.getFacetResults(), null);      
     }
 
+    List<FacetRequest> ddRequests = new ArrayList<FacetRequest>();
+    for(FacetRequest fr : fsp.facetRequests) {
+      assert fr.categoryPath.length > 0;
+      if (!drillDownDims.containsKey(fr.categoryPath.components[0])) {
+        ddRequests.add(fr);
+      }
+    }
+    FacetSearchParams fsp2;
+    if (!ddRequests.isEmpty()) {
+      fsp2 = new FacetSearchParams(fsp.indexingParams, ddRequests);
+    } else {
+      fsp2 = null;
+    }
+
     BooleanQuery ddq = query.getBooleanQuery();
     BooleanClause[] clauses = ddq.getClauses();
 
@@ -173,7 +195,7 @@ public class DrillSideways {
       startClause = 1;
     }
 
-    FacetsCollector drillDownCollector = FacetsCollector.create(getDrillDownAccumulator(fsp));
+    FacetsCollector drillDownCollector = fsp2 == null ? null : FacetsCollector.create(getDrillDownAccumulator(fsp2));
 
     FacetsCollector[] drillSidewaysCollectors = new FacetsCollector[drillDownDims.size()];
 
@@ -225,6 +247,8 @@ public class DrillSideways {
               break;
             }
           }
+        } else {
+          useCollectorMethod = true;
         }
       }
     }
@@ -246,6 +270,7 @@ public class DrillSideways {
 
     List<FacetResult> mergedResults = new ArrayList<FacetResult>();
     int[] requestUpto = new int[drillDownDims.size()];
+    int ddUpto = 0;
     for(int i=0;i<fsp.facetRequests.size();i++) {
       FacetRequest fr = fsp.facetRequests.get(i);
       assert fr.categoryPath.length > 0;
@@ -260,7 +285,7 @@ public class DrillSideways {
           //System.out.println("get DD results");
         }
         //System.out.println("add dd results " + i);
-        mergedResults.add(drillDownResults.get(i));
+        mergedResults.add(drillDownResults.get(ddUpto++));
       } else {
         // Drill sideways dim:
         int dim = dimIndex.intValue();
@@ -359,7 +384,7 @@ public class DrillSideways {
 
     subQuery.setMinimumNumberShouldMatch(minShouldMatch);
 
-    //System.out.println("EXE " + topQuery);
+    // System.out.println("EXE " + topQuery);
 
     // Collects against the passed-in
     // drillDown/SidewaysCollectors as a side effect:

Modified: lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysCollector.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysCollector.java (original)
+++ lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysCollector.java Mon May 13 13:24:00 2013
@@ -82,7 +82,9 @@ class DrillSidewaysCollector extends Col
       // drillDown collector:
       //System.out.println("  hit " + drillDownCollector);
       hitCollector.collect(doc);
-      drillDownCollector.collect(doc);
+      if (drillDownCollector != null) {
+        drillDownCollector.collect(doc);
+      }
 
       // Also collect across all drill-sideways counts so
       // we "merge in" drill-down counts for this
@@ -98,21 +100,28 @@ class DrillSidewaysCollector extends Col
       }
 
     } else {
+      boolean found = false;
       for(int i=0;i<subScorers.length;i++) {
         if (subScorers[i] == null) {
           // This segment did not have any docs with this
           // drill-down field & value:
-          continue;
+          drillSidewaysCollectors[i].collect(doc);
+          assert allMatchesFrom(i+1, doc);
+          found = true;
+          break;
         }
         int subDoc = subScorers[i].docID();
-        //System.out.println("  sub: " + subDoc);
+        //System.out.println("  i=" + i + " sub: " + subDoc);
         if (subDoc != doc) {
+          //System.out.println("  +ds[" + i + "]");
           assert subDoc > doc: "subDoc=" + subDoc + " doc=" + doc;
           drillSidewaysCollectors[i].collect(doc);
           assert allMatchesFrom(i+1, doc);
+          found = true;
           break;
         }
       }
+      assert found;
     }
   }
 
@@ -134,8 +143,11 @@ class DrillSidewaysCollector extends Col
 
   @Override
   public void setNextReader(AtomicReaderContext leaf) throws IOException {
+    //System.out.println("DS.setNextReader reader=" + leaf.reader());
     hitCollector.setNextReader(leaf);
-    drillDownCollector.setNextReader(leaf);
+    if (drillDownCollector != null) {
+      drillDownCollector.setNextReader(leaf);
+    }
     for(Collector dsc : drillSidewaysCollectors) {
       dsc.setNextReader(leaf);
     }
@@ -166,7 +178,9 @@ class DrillSidewaysCollector extends Col
     Arrays.fill(subScorers, null);
     findScorers(scorer);
     hitCollector.setScorer(scorer);
-    drillDownCollector.setScorer(scorer);
+    if (drillDownCollector != null) {
+      drillDownCollector.setScorer(scorer);
+    }
     for(Collector dsc : drillSidewaysCollectors) {
       dsc.setScorer(scorer);
     }

Modified: lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysScorer.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysScorer.java (original)
+++ lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysScorer.java Mon May 13 13:24:00 2013
@@ -63,8 +63,10 @@ class DrillSidewaysScorer extends Scorer
     //}
     //System.out.println("score r=" + context.reader());
     collector.setScorer(this);
-    drillDownCollector.setScorer(this);
-    drillDownCollector.setNextReader(context);
+    if (drillDownCollector != null) {
+      drillDownCollector.setScorer(this);
+      drillDownCollector.setNextReader(context);
+    }
     for(DocsEnumsAndFreq dim : dims) {
       dim.sidewaysCollector.setScorer(this);
       dim.sidewaysCollector.setNextReader(context);
@@ -393,7 +395,9 @@ class DrillSidewaysScorer extends Scorer
     //}
 
     collector.collect(collectDocID);
-    drillDownCollector.collect(collectDocID);
+    if (drillDownCollector != null) {
+      drillDownCollector.collect(collectDocID);
+    }
 
     // TODO: we could "fix" faceting of the sideways counts
     // to do this "union" (of the drill down hits) in the

Modified: lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java (original)
+++ lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java Mon May 13 13:24:00 2013
@@ -196,7 +196,7 @@ public class FacetsAccumulator {
     return res;
   }
 
-  protected boolean requiresDocScores() {
+  public boolean requiresDocScores() {
     return getAggregator().requiresDocScores();
   }
 }

Modified: lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeAccumulator.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeAccumulator.java (original)
+++ lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeAccumulator.java Mon May 13 13:24:00 2013
@@ -17,7 +17,10 @@ package org.apache.lucene.facet.range;
  * limitations under the License.
  */
 
+import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.DoubleDocValuesField;
@@ -29,18 +32,30 @@ import org.apache.lucene.document.LongFi
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.facet.FacetTestCase;
 import org.apache.lucene.facet.FacetTestUtils;
+import org.apache.lucene.facet.index.FacetFields;
 import org.apache.lucene.facet.params.FacetIndexingParams;
 import org.apache.lucene.facet.params.FacetSearchParams;
+import org.apache.lucene.facet.search.CountFacetRequest;
 import org.apache.lucene.facet.search.DrillDownQuery;
+import org.apache.lucene.facet.search.DrillSideways.DrillSidewaysResult;
+import org.apache.lucene.facet.search.DrillSideways;
+import org.apache.lucene.facet.search.FacetRequest;
 import org.apache.lucene.facet.search.FacetResult;
 import org.apache.lucene.facet.search.FacetResultNode;
+import org.apache.lucene.facet.search.FacetsAccumulator;
 import org.apache.lucene.facet.search.FacetsCollector;
+import org.apache.lucene.facet.taxonomy.CategoryPath;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
+import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.NumericRangeQuery;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util._TestUtil;
 
 public class TestRangeAccumulator extends FacetTestCase {
@@ -81,6 +96,114 @@ public class TestRangeAccumulator extend
     d.close();
   }
 
+  /** Tests single request that mixes Range and non-Range
+   *  faceting, with DrillSideways. */
+  public void testMixedRangeAndNonRange() throws Exception {
+    Directory d = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), d);
+    Directory td = newDirectory();
+    DirectoryTaxonomyWriter tw = new DirectoryTaxonomyWriter(td, IndexWriterConfig.OpenMode.CREATE);
+    FacetFields ff = new FacetFields(tw);
+
+    for(long l=0;l<100;l++) {
+      Document doc = new Document();
+      // For computing range facet counts:
+      doc.add(new NumericDocValuesField("field", l));
+      // For drill down by numeric range:
+      doc.add(new LongField("field", l, Field.Store.NO));
+
+      CategoryPath cp;
+      if ((l&3) == 0) {
+        cp = new CategoryPath("dim", "a");
+      } else {
+        cp = new CategoryPath("dim", "b");
+      }
+      ff.addFields(doc, Collections.singletonList(cp));
+      w.addDocument(doc);
+    }
+
+    IndexReader r = w.getReader();
+    w.close();
+
+    final TaxonomyReader tr = new DirectoryTaxonomyReader(tw);
+    tw.close();
+
+    IndexSearcher s = newSearcher(r);
+
+    final FacetSearchParams fsp = new FacetSearchParams(
+                                new CountFacetRequest(new CategoryPath("dim"), 2),
+                                new RangeFacetRequest<LongRange>("field",
+                                                      new LongRange("less than 10", 0L, true, 10L, false),
+                                                      new LongRange("less than or equal to 10", 0L, true, 10L, true),
+                                                      new LongRange("over 90", 90L, false, 100L, false),
+                                                      new LongRange("90 or above", 90L, true, 100L, false),
+                                                      new LongRange("over 1000", 1000L, false, Long.MAX_VALUE, false)));
+
+    final Set<String> dimSeen = new HashSet<String>();
+
+    DrillSideways ds = new DrillSideways(s, tr) {
+        @Override
+        protected FacetsAccumulator getDrillDownAccumulator(FacetSearchParams fsp) {
+          checkSeen(fsp);
+          return RangeFacetsAccumulatorWrapper.create(fsp, searcher.getIndexReader(), tr);
+        }
+
+        @Override
+        protected FacetsAccumulator getDrillSidewaysAccumulator(String dim, FacetSearchParams fsp) {
+          checkSeen(fsp);
+          return RangeFacetsAccumulatorWrapper.create(fsp, searcher.getIndexReader(), tr);
+        }
+
+        private void checkSeen(FacetSearchParams fsp) {
+          // Each dim should should up only once, across
+          // both drillDown and drillSideways requests:
+          for(FacetRequest fr : fsp.facetRequests) {
+            String dim = fr.categoryPath.components[0];
+            assertFalse("dim " + dim + " already seen", dimSeen.contains(dim));
+            dimSeen.add(dim);
+          }
+        }
+
+        @Override
+        protected boolean scoreSubDocsAtOnce() {
+          return random().nextBoolean();
+        }
+      };
+
+    // First search, no drill downs:
+    DrillDownQuery ddq = new DrillDownQuery(FacetIndexingParams.DEFAULT, new MatchAllDocsQuery());
+    DrillSidewaysResult dsr = ds.search(null, ddq, 10, fsp);
+
+    assertEquals(100, dsr.hits.totalHits);
+    assertEquals(2, dsr.facetResults.size());
+    assertEquals("dim (0)\n  b (75)\n  a (25)\n", FacetTestUtils.toSimpleString(dsr.facetResults.get(0)));
+    assertEquals("field (0)\n  less than 10 (10)\n  less than or equal to 10 (11)\n  over 90 (9)\n  90 or above (10)\n  over 1000 (0)\n", FacetTestUtils.toSimpleString(dsr.facetResults.get(1)));
+
+    // Second search, drill down on dim=b:
+    ddq = new DrillDownQuery(FacetIndexingParams.DEFAULT, new MatchAllDocsQuery());
+    ddq.add(new CategoryPath("dim", "b"));
+    dimSeen.clear();
+    dsr = ds.search(null, ddq, 10, fsp);
+
+    assertEquals(75, dsr.hits.totalHits);
+    assertEquals(2, dsr.facetResults.size());
+    assertEquals("dim (0)\n  b (75)\n  a (25)\n", FacetTestUtils.toSimpleString(dsr.facetResults.get(0)));
+    assertEquals("field (0)\n  less than 10 (7)\n  less than or equal to 10 (8)\n  over 90 (7)\n  90 or above (8)\n  over 1000 (0)\n", FacetTestUtils.toSimpleString(dsr.facetResults.get(1)));
+
+    // Third search, drill down on "less than or equal to 10":
+    ddq = new DrillDownQuery(FacetIndexingParams.DEFAULT, new MatchAllDocsQuery());
+    ddq.add("field", NumericRangeQuery.newLongRange("field", 0L, 10L, true, true));
+    dimSeen.clear();
+    dsr = ds.search(null, ddq, 10, fsp);
+
+    assertEquals(11, dsr.hits.totalHits);
+    assertEquals(2, dsr.facetResults.size());
+    assertEquals("dim (0)\n  b (8)\n  a (3)\n", FacetTestUtils.toSimpleString(dsr.facetResults.get(0)));
+    assertEquals("field (0)\n  less than 10 (10)\n  less than or equal to 10 (11)\n  over 90 (9)\n  90 or above (10)\n  over 1000 (0)\n", FacetTestUtils.toSimpleString(dsr.facetResults.get(1)));
+
+    IOUtils.close(tr, td, r, d);
+  }
+
   public void testBasicDouble() throws Exception {
     Directory d = newDirectory();
     RandomIndexWriter w = new RandomIndexWriter(random(), d);

Modified: lucene/dev/branches/lucene4258/lucene/module-build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/module-build.xml?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/module-build.xml (original)
+++ lucene/dev/branches/lucene4258/lucene/module-build.xml Mon May 13 13:24:00 2013
@@ -220,7 +220,29 @@
     </ant>
     <property name="facet-javadocs.uptodate" value="true"/>
   </target>
- 
+
+  <property name="replicator.jar" value="${common.dir}/build/replicator/lucene-replicator-${version}.jar"/>
+  <target name="check-replicator-uptodate" unless="replicator.uptodate">
+    <module-uptodate name="replicator" jarfile="${replicator.jar}" property="replicator.uptodate"/>
+  </target>
+  <target name="jar-replicator" unless="replicator.uptodate" depends="check-replicator-uptodate">
+    <ant dir="${common.dir}/replicator" target="jar-core" inheritall="false">
+      <propertyset refid="uptodate.and.compiled.properties"/>
+    </ant>
+    <property name="replicator.uptodate" value="true"/>
+  </target>
+
+  <property name="replicator-javadoc.jar" value="${common.dir}/build/replicator/lucene-replicator-${version}-javadoc.jar"/>
+  <target name="check-replicator-javadocs-uptodate" unless="replicator-javadocs.uptodate">
+    <module-uptodate name="replicator" jarfile="${replicator-javadoc.jar}" property="replicator-javadocs.uptodate"/>
+  </target>
+  <target name="javadocs-replicator" unless="replicator-javadocs.uptodate" depends="check-replicator-javadocs-uptodate">
+    <ant dir="${common.dir}/replicator" target="javadocs" inheritAll="false">
+      <propertyset refid="uptodate.and.compiled.properties"/>
+    </ant>
+    <property name="replicator-javadocs.uptodate" value="true"/>
+  </target>
+
   <property name="analyzers-icu.jar" value="${common.dir}/build/analysis/icu/lucene-analyzers-icu-${version}.jar"/>
   <target name="check-analyzers-icu-uptodate" unless="analyzers-icu.uptodate">
     <module-uptodate name="analysis/icu" jarfile="${analyzers-icu.jar}" property="analyzers-icu.uptodate"/>

Modified: lucene/dev/branches/lucene4258/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java?rev=1481840&r1=1481839&r2=1481840&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java (original)
+++ lucene/dev/branches/lucene4258/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java Mon May 13 13:24:00 2013
@@ -572,24 +572,53 @@ public abstract class QueryParserBase im
       if (severalTokensAtSamePosition || (!quoted && !autoGeneratePhraseQueries)) {
         if (positionCount == 1 || (!quoted && !autoGeneratePhraseQueries)) {
           // no phrase query:
-          BooleanQuery q = newBooleanQuery(positionCount == 1);
-
-          BooleanClause.Occur occur = positionCount > 1 && operator == AND_OPERATOR ?
-            BooleanClause.Occur.MUST : BooleanClause.Occur.SHOULD;
-
-          for (int i = 0; i < numTokens; i++) {
-            try {
-              boolean hasNext = buffer.incrementToken();
-              assert hasNext == true;
-              termAtt.fillBytesRef();
-            } catch (IOException e) {
-              // safe to ignore, because we know the number of tokens
+          
+          if (positionCount == 1) {
+            // simple case: only one position, with synonyms
+            BooleanQuery q = newBooleanQuery(true);
+            for (int i = 0; i < numTokens; i++) {
+              try {
+                boolean hasNext = buffer.incrementToken();
+                assert hasNext == true;
+                termAtt.fillBytesRef();
+              } catch (IOException e) {
+                // safe to ignore, because we know the number of tokens
+              }
+              Query currentQuery = newTermQuery(
+                  new Term(field, BytesRef.deepCopyOf(bytes)));
+              q.add(currentQuery, BooleanClause.Occur.SHOULD);
+            }
+            return q;
+          } else {
+            // multiple positions
+            BooleanQuery q = newBooleanQuery(false);
+            final BooleanClause.Occur occur = operator == Operator.AND ? BooleanClause.Occur.MUST : BooleanClause.Occur.SHOULD;
+            Query currentQuery = null;
+            for (int i = 0; i < numTokens; i++) {
+              try {
+                boolean hasNext = buffer.incrementToken();
+                assert hasNext == true;
+                termAtt.fillBytesRef();
+              } catch (IOException e) {
+                // safe to ignore, because we know the number of tokens
+              }
+              if (posIncrAtt != null && posIncrAtt.getPositionIncrement() == 0) {
+                if (!(currentQuery instanceof BooleanQuery)) {
+                  Query t = currentQuery;
+                  currentQuery = newBooleanQuery(true);
+                  ((BooleanQuery)currentQuery).add(t, BooleanClause.Occur.SHOULD);
+                }
+                ((BooleanQuery)currentQuery).add(newTermQuery(new Term(field, BytesRef.deepCopyOf(bytes))), BooleanClause.Occur.SHOULD);
+              } else {
+                if (currentQuery != null) {
+                  q.add(currentQuery, occur);
+                }
+                currentQuery = newTermQuery(new Term(field, BytesRef.deepCopyOf(bytes)));
+              }
             }
-            Query currentQuery = newTermQuery(
-                new Term(field, BytesRef.deepCopyOf(bytes)));
             q.add(currentQuery, occur);
+            return q;
           }
-          return q;
         }
         else {
           // phrase query: