You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2011/05/23 18:12:04 UTC

svn commit: r1126565 [2/3] - in /lucene/dev/branches/flexscoring/lucene: ./ backwards/ contrib/ contrib/ant/ contrib/db/ contrib/lucli/ contrib/misc/src/java/org/apache/lucene/index/ contrib/misc/src/java/org/apache/lucene/store/ contrib/misc/src/test/...

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentReader.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentReader.java Mon May 23 16:12:02 2011
@@ -33,11 +33,11 @@ import org.apache.lucene.index.codecs.Fi
 import org.apache.lucene.store.BufferedIndexInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.BitVector;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CloseableThreadLocal;
+import org.apache.lucene.util.StringHelper;
 
 /**
  * @lucene.experimental
@@ -67,8 +67,8 @@ public class SegmentReader extends Index
   private int rollbackPendingDeleteCount;
 
   // optionally used for the .nrm file shared by multiple norms
-  private IndexInput singleNormStream;
-  private AtomicInteger singleNormRef;
+  IndexInput singleNormStream;
+  AtomicInteger singleNormRef;
 
   CoreReaders core;
 
@@ -249,225 +249,9 @@ public class SegmentReader extends Index
     }
   }
   
-  /**
-   * Byte[] referencing is used because a new norm object needs 
-   * to be created for each clone, and the byte array is all 
-   * that is needed for sharing between cloned readers.  The 
-   * current norm referencing is for sharing between readers 
-   * whereas the byte[] referencing is for copy on write which 
-   * is independent of reader references (i.e. incRef, decRef).
-   */
-
-  final class Norm implements Cloneable {
-    private int refCount = 1;
-
-    // If this instance is a clone, the originalNorm
-    // references the Norm that has a real open IndexInput:
-    private Norm origNorm;
-
-    private IndexInput in;
-    private long normSeek;
-
-    // null until bytes is set
-    private AtomicInteger bytesRef;
-    private byte[] bytes;
-    private boolean dirty;
-    private int number;
-    private boolean rollbackDirty;
-    private long sum;
-    
-    public Norm(IndexInput in, int number, long normSeek) {
-      this.in = in;
-      this.number = number;
-      this.normSeek = normSeek;
-    }
-
-    public synchronized void incRef() {
-      assert refCount > 0 && (origNorm == null || origNorm.refCount > 0);
-      refCount++;
-    }
-
-    private void closeInput() throws IOException {
-      if (in != null) {
-        if (in != singleNormStream) {
-          // It's private to us -- just close it
-          in.close();
-        } else {
-          // We are sharing this with others -- decRef and
-          // maybe close the shared norm stream
-          if (singleNormRef.decrementAndGet() == 0) {
-            singleNormStream.close();
-            singleNormStream = null;
-          }
-        }
-
-        in = null;
-      }
-    }
-
-    public synchronized void decRef() throws IOException {
-      assert refCount > 0 && (origNorm == null || origNorm.refCount > 0);
-
-      if (--refCount == 0) {
-        if (origNorm != null) {
-          origNorm.decRef();
-          origNorm = null;
-        } else {
-          closeInput();
-        }
-
-        if (bytes != null) {
-          assert bytesRef != null;
-          bytesRef.decrementAndGet();
-          bytes = null;
-          bytesRef = null;
-        } else {
-          assert bytesRef == null;
-        }
-      }
-    }
-
-    // Load & cache full bytes array.  Returns bytes.
-    public synchronized byte[] bytes() throws IOException {
-      assert refCount > 0 && (origNorm == null || origNorm.refCount > 0);
-      if (bytes == null) {                     // value not yet read
-        assert bytesRef == null;
-        if (origNorm != null) {
-          // Ask origNorm to load so that for a series of
-          // reopened readers we share a single read-only
-          // byte[]
-          bytes = origNorm.bytes();
-          bytesRef = origNorm.bytesRef;
-          bytesRef.incrementAndGet();
-          sum = origNorm.sum;
-
-          // Once we've loaded the bytes we no longer need
-          // origNorm:
-          origNorm.decRef();
-          origNorm = null;
-
-        } else {
-          // We are the origNorm, so load the bytes for real
-          // ourself:
-          final int count = maxDoc();
-          bytes = new byte[count];
-
-          // Since we are orig, in must not be null
-          assert in != null;
-
-          // Read from disk.
-          synchronized(in) {
-            in.seek(normSeek);
-            in.readBytes(bytes, 0, count, false);
-            // nocommit: version the file, and add this sum.
-            sum = 0;
-            for (int i = 0; i < count; i++) {
-              sum += (bytes[i] & 0xff);
-            }
-          }
-
-          bytesRef = new AtomicInteger(1);
-          closeInput();
-        }
-      }
-
-      return bytes;
-    }
-
-    // Only for testing
-    AtomicInteger bytesRef() {
-      return bytesRef;
-    }
-
-    // Called if we intend to change a norm value.  We make a
-    // private copy of bytes if it's shared with others:
-    public synchronized byte[] copyOnWrite() throws IOException {
-      assert refCount > 0 && (origNorm == null || origNorm.refCount > 0);
-      bytes();
-      assert bytes != null;
-      assert bytesRef != null;
-      if (bytesRef.get() > 1) {
-        // I cannot be the origNorm for another norm
-        // instance if I'm being changed.  Ie, only the
-        // "head Norm" can be changed:
-        assert refCount == 1;
-        final AtomicInteger oldRef = bytesRef;
-        bytes = cloneNormBytes(bytes);
-        bytesRef = new AtomicInteger(1);
-        oldRef.decrementAndGet();
-      }
-      dirty = true;
-      return bytes;
-    }
-    
-    // Returns a copy of this Norm instance that shares
-    // IndexInput & bytes with the original one
-    @Override
-    public synchronized Object clone() {
-      assert refCount > 0 && (origNorm == null || origNorm.refCount > 0);
-        
-      Norm clone;
-      try {
-        clone = (Norm) super.clone();
-      } catch (CloneNotSupportedException cnse) {
-        // Cannot happen
-        throw new RuntimeException("unexpected CloneNotSupportedException", cnse);
-      }
-      clone.refCount = 1;
-
-      if (bytes != null) {
-        assert bytesRef != null;
-        assert origNorm == null;
-
-        // Clone holds a reference to my bytes:
-        clone.bytesRef.incrementAndGet();
-      } else {
-        assert bytesRef == null;
-        if (origNorm == null) {
-          // I become the origNorm for the clone:
-          clone.origNorm = this;
-        }
-        clone.origNorm.incRef();
-      }
-
-      // Only the origNorm will actually readBytes from in:
-      clone.in = null;
-
-      return clone;
-    }
 
-    // Flush all pending changes to the next generation
-    // separate norms file.
-    public void reWrite(SegmentInfo si) throws IOException {
-      assert refCount > 0 && (origNorm == null || origNorm.refCount > 0): "refCount=" + refCount + " origNorm=" + origNorm;
 
-      // NOTE: norms are re-written in regular directory, not cfs
-      si.advanceNormGen(this.number);
-      final String normFileName = si.getNormFileName(this.number);
-      IndexOutput out = directory().createOutput(normFileName);
-      boolean success = false;
-      try {
-        try {
-          out.writeBytes(bytes, maxDoc());
-        } finally {
-          out.close();
-        }
-        success = true;
-      } finally {
-        if (!success) {
-          try {
-            directory().deleteFile(normFileName);
-          } catch (Throwable t) {
-            // suppress this so we keep throwing the
-            // original exception
-          }
-        }
-      }
-      this.dirty = false;
-    }
-  }
-
-  Map<String,Norm> norms = new HashMap<String,Norm>();
+  Map<String,SegmentNorms> norms = new HashMap<String,SegmentNorms>();
   
   /**
    * @throws CorruptIndexException if the index is corrupt
@@ -665,16 +449,16 @@ public class SegmentReader extends Index
         }
       }
 
-      clone.norms = new HashMap<String,Norm>();
+      clone.norms = new HashMap<String,SegmentNorms>();
 
       // Clone norms
       for (FieldInfo fi : core.fieldInfos) {
         // Clone unchanged norms to the cloned reader
         if (doClone || !fieldNormsChanged.contains(fi.number)) {
           final String curField = fi.name;
-          Norm norm = this.norms.get(curField);
+          SegmentNorms norm = this.norms.get(curField);
           if (norm != null)
-            clone.norms.put(curField, (Norm) norm.clone());
+            clone.norms.put(curField, (SegmentNorms) norm.clone());
         }
       }
 
@@ -744,7 +528,7 @@ public class SegmentReader extends Index
 
     if (normsDirty) {               // re-write norms
       si.initNormGen();
-      for (final Norm norm : norms.values()) {
+      for (final SegmentNorms norm : norms.values()) {
         if (norm.dirty) {
           norm.reWrite(si);
         }
@@ -770,7 +554,7 @@ public class SegmentReader extends Index
       deletedDocs = null;
     }
 
-    for (final Norm norm : norms.values()) {
+    for (final SegmentNorms norm : norms.values()) {
       norm.decRef();
     }
     if (core != null) {
@@ -940,7 +724,7 @@ public class SegmentReader extends Index
   @Override
   public byte[] norms(String field) throws IOException {
     ensureOpen();
-    final Norm norm = norms.get(field);
+    final SegmentNorms norm = norms.get(field);
     if (norm == null) {
       // not indexed, or norms not stored
       return null;  
@@ -951,7 +735,7 @@ public class SegmentReader extends Index
   @Override
   public synchronized long getSumOfNorms(String field) throws IOException {
     ensureOpen();
-    Norm norm = norms.get(field);
+    SegmentNorms norm = norms.get(field);
     if (norm == null) return 0; // not indexed, or norms not stored
     norm.bytes(); // load norms if not loaded
     return norm.sum;
@@ -960,7 +744,7 @@ public class SegmentReader extends Index
   @Override
   protected void doSetNorm(int doc, String field, byte value)
           throws IOException {
-    Norm norm = norms.get(field);
+    SegmentNorms norm = norms.get(field);
     if (norm == null)                             // not an indexed field
       return;
 
@@ -1005,11 +789,23 @@ public class SegmentReader extends Index
           // If this were to change in the future, a clone could be done here.
           normInput = singleNormStream;
         } else {
-          normSeek = 0;
           normInput = d.openInput(fileName);
+          // if the segment was created in 3.2 or after, we wrote the header for sure,
+          // and don't need to do the sketchy file size check. otherwise, we check 
+          // if the size is exactly equal to maxDoc to detect a headerless file.
+          // NOTE: remove this check in Lucene 5.0!
+          String version = si.getVersion();
+          final boolean isUnversioned = 
+            (version == null || StringHelper.getVersionComparator().compare(version, "3.2") < 0)
+            && normInput.length() == maxDoc();
+          if (isUnversioned) {
+            normSeek = 0;
+          } else {
+            normSeek = SegmentMerger.NORMS_HEADER.length;
+          }
         }
 
-        norms.put(fi.name, new Norm(normInput, fi.number, normSeek));
+        norms.put(fi.name, new SegmentNorms(normInput, fi.number, normSeek, this));
         nextNormSeek += maxDoc; // increment also if some norms are separate
       }
     }
@@ -1029,7 +825,7 @@ public class SegmentReader extends Index
     if (singleNormStream != null) {
       return false;
     }
-    for (final Norm norm : norms.values()) {
+    for (final SegmentNorms norm : norms.values()) {
       if (norm.refCount > 0) {
         return false;
       }
@@ -1176,7 +972,7 @@ public class SegmentReader extends Index
     rollbackDeletedDocsDirty = deletedDocsDirty;
     rollbackNormsDirty = normsDirty;
     rollbackPendingDeleteCount = pendingDeleteCount;
-    for (Norm norm : norms.values()) {
+    for (SegmentNorms norm : norms.values()) {
       norm.rollbackDirty = norm.dirty;
     }
   }
@@ -1187,7 +983,7 @@ public class SegmentReader extends Index
     deletedDocsDirty = rollbackDeletedDocsDirty;
     normsDirty = rollbackNormsDirty;
     pendingDeleteCount = rollbackPendingDeleteCount;
-    for (Norm norm : norms.values()) {
+    for (SegmentNorms norm : norms.values()) {
       norm.dirty = norm.rollbackDirty;
     }
   }

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java Mon May 23 16:12:02 2011
@@ -251,9 +251,7 @@ public class TieredMergePolicy extends M
     final Collection<SegmentInfo> merging = writer.get().getMergingSegments();
     final Collection<SegmentInfo> toBeMerged = new HashSet<SegmentInfo>();
 
-    final List<SegmentInfo> infosSorted = new ArrayList<SegmentInfo>();
-    infosSorted.addAll(infos);
-
+    final List<SegmentInfo> infosSorted = new ArrayList<SegmentInfo>(infos.asList());
     Collections.sort(infosSorted, segmentByteSizeDescending);
 
     // Compute total index bytes & print details about the index

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexWriter.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexWriter.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexWriter.java Mon May 23 16:12:02 2011
@@ -200,6 +200,7 @@ public class VariableGapTermsIndexWriter
   private class FSTFieldWriter extends FieldWriter {
     private final Builder<Long> fstBuilder;
     private final PositiveIntOutputs fstOutputs;
+    private final long startTermsFilePointer;
 
     final FieldInfo fieldInfo;
     int numIndexTerms;
@@ -220,6 +221,7 @@ public class VariableGapTermsIndexWriter
 
       // Always put empty string in
       fstBuilder.add(new BytesRef(), fstOutputs.get(termsFilePointer));
+      startTermsFilePointer = termsFilePointer;
     }
 
     @Override
@@ -239,6 +241,11 @@ public class VariableGapTermsIndexWriter
 
     @Override
     public void add(BytesRef text, TermStats stats, long termsFilePointer) throws IOException {
+      if (text.length == 0) {
+        // We already added empty string in ctor
+        assert termsFilePointer == startTermsFilePointer;
+        return;
+      }
       final int lengthSave = text.length;
       text.length = indexedTermPrefixLength(lastTerm, text);
       try {

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/DocIdSetIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/DocIdSetIterator.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/DocIdSetIterator.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/DocIdSetIterator.java Mon May 23 16:12:02 2011
@@ -61,9 +61,10 @@ public abstract class DocIdSetIterator {
   public abstract int nextDoc() throws IOException;
 
   /**
-   * Advances to the first beyond the current whose document number is greater
-   * than or equal to <i>target</i>. Returns the current document number or
-   * {@link #NO_MORE_DOCS} if there are no more docs in the set.
+   * Advances to the first beyond (see NOTE below) the current whose document
+   * number is greater than or equal to <i>target</i>. Returns the current
+   * document number or {@link #NO_MORE_DOCS} if there are no more docs in the
+   * set.
    * <p>
    * Behaves as if written:
    * 
@@ -78,7 +79,7 @@ public abstract class DocIdSetIterator {
    * 
    * Some implementations are considerably more efficient than that.
    * <p>
-   * <b>NOTE:</b> when <code> target &le; current</code> implementations may opt 
+   * <b>NOTE:</b> when <code> target &le; current</code> implementations may opt
    * not to advance beyond their current {@link #docID()}.
    * <p>
    * <b>NOTE:</b> this method may be called with {@link #NO_MORE_DOCS} for

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java Mon May 23 16:12:02 2011
@@ -127,7 +127,7 @@ public class FieldCacheTermsFilter exten
 
     public FieldCacheTermsFilterDocIdSet(FieldCache.DocTermsIndex fcsi) {
       this.fcsi = fcsi;
-      openBitSet = new OpenBitSet(this.fcsi.size());
+      openBitSet = new OpenBitSet(this.fcsi.numOrd());
       final BytesRef spare = new BytesRef();
       for (int i=0;i<terms.length;i++) {
         int termNumber = this.fcsi.binarySearchLookup(terms[i], spare);

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/SloppyPhraseScorer.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/SloppyPhraseScorer.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/SloppyPhraseScorer.java Mon May 23 16:12:02 2011
@@ -18,7 +18,7 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.util.HashMap;
+import java.util.HashSet;
 
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 
@@ -113,8 +113,14 @@ final class SloppyPhraseScorer extends P
 
     /**
      * Init PhrasePositions in place.
-     * There is a one time initialization for this scorer:
+     * There is a one time initialization for this scorer (taking place at the first doc that matches all terms):
      * <br>- Put in repeats[] each pp that has another pp with same position in the doc.
+     *       This relies on that the position in PP is computed as (TP.position - offset) and 
+     *       so by adding offset we actually compare positions and identify that the two are 
+     *       the same term.
+     *       An exclusion to this is two distinct terms in the same offset in query and same 
+     *       position in doc. This case is detected by comparing just the (query) offsets, 
+     *       and two such PPs are not considered "repeating". 
      * <br>- Also mark each such pp by pp.repeats = true.
      * <br>Later can consult with repeats[] in termPositionsDiffer(pp), making that check efficient.
      * In particular, this allows to score queries with no repetitions with no overhead due to this computation.
@@ -149,23 +155,26 @@ final class SloppyPhraseScorer extends P
         if (!checkedRepeats) {
             checkedRepeats = true;
             // check for repeats
-            HashMap<PhrasePositions, Object> m = null;
+            HashSet<PhrasePositions> m = null;
             for (PhrasePositions pp = first; pp != null; pp = pp.next) {
                 int tpPos = pp.position + pp.offset;
                 for (PhrasePositions pp2 = pp.next; pp2 != null; pp2 = pp2.next) {
+                    if (pp.offset == pp2.offset) {
+                      continue; // not a repetition: the two PPs are originally in same offset in the query! 
+                    }
                     int tpPos2 = pp2.position + pp2.offset;
                     if (tpPos2 == tpPos) { 
                         if (m == null)
-                            m = new HashMap<PhrasePositions, Object>();
+                            m = new HashSet<PhrasePositions>();
                         pp.repeats = true;
                         pp2.repeats = true;
-                        m.put(pp,null);
-                        m.put(pp2,null);
+                        m.add(pp);
+                        m.add(pp2);
                     }
                 }
             }
             if (m!=null)
-                repeats = m.keySet().toArray(new PhrasePositions[0]);
+                repeats = m.toArray(new PhrasePositions[0]);
         }
         
         // with repeats must advance some repeating pp's so they all start with differing tp's       
@@ -208,11 +217,16 @@ final class SloppyPhraseScorer extends P
         int tpPos = pp.position + pp.offset;
         for (int i = 0; i < repeats.length; i++) {
             PhrasePositions pp2 = repeats[i];
-            if (pp2 == pp)
+            if (pp2 == pp) {
                 continue;
+            }
+            if (pp.offset == pp2.offset) {
+              continue; // not a repetition: the two PPs are originally in same offset in the query! 
+            }
             int tpPos2 = pp2.position + pp2.offset;
-            if (tpPos2 == tpPos)
+            if (tpPos2 == tpPos) {
                 return pp.offset > pp2.offset ? pp : pp2; // do not differ: return the one with higher offset.
+            }
         }
         return null; 
     }

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/Directory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/Directory.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/Directory.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/Directory.java Mon May 23 16:12:02 2011
@@ -65,10 +65,6 @@ public abstract class Directory implemen
   public abstract long fileModified(String name)
        throws IOException;
 
-  /** Set the modified time of an existing file to now. */
-  public abstract void touchFile(String name)
-       throws IOException;
-
   /** Removes an existing file in the directory. */
   public abstract void deleteFile(String name)
        throws IOException;

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/FSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/FSDirectory.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/FSDirectory.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/FSDirectory.java Mon May 23 16:12:02 2011
@@ -272,14 +272,6 @@ public abstract class FSDirectory extend
     return file.lastModified();
   }
 
-  /** Set the modified time of an existing file to now. */
-  @Override
-  public void touchFile(String name) {
-    ensureOpen();
-    File file = new File(directory, name);
-    file.setLastModified(System.currentTimeMillis());
-  }
-
   /** Returns the length in bytes of a file in the directory. */
   @Override
   public long fileLength(String name) throws IOException {

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java Mon May 23 16:12:02 2011
@@ -115,11 +115,6 @@ public class FileSwitchDirectory extends
   }
 
   @Override
-  public void touchFile(String name) throws IOException {
-    getDirectory(name).touchFile(name);
-  }
-
-  @Override
   public void deleteFile(String name) throws IOException {
     getDirectory(name).deleteFile(name);
   }

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java Mon May 23 16:12:02 2011
@@ -172,7 +172,7 @@ public class NIOFSDirectory extends FSDi
         final OutOfMemoryError outOfMemoryError = new OutOfMemoryError(
               "OutOfMemoryError likely caused by the Sun VM Bug described in "
               + "https://issues.apache.org/jira/browse/LUCENE-1566; try calling FSDirectory.setReadChunkSize "
-              + "with a a value smaller than the current chunk size (" + chunkSize + ")");
+              + "with a value smaller than the current chunk size (" + chunkSize + ")");
         outOfMemoryError.initCause(e);
         throw outOfMemoryError;
       }

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/RAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/RAMDirectory.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/RAMDirectory.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/RAMDirectory.java Mon May 23 16:12:02 2011
@@ -27,8 +27,6 @@ import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
 
-import org.apache.lucene.util.ThreadInterruptedException;
-
 /**
  * A memory-resident {@link Directory} implementation.  Locking
  * implementation is by default the {@link SingleInstanceLockFactory}
@@ -112,30 +110,6 @@ public class RAMDirectory extends Direct
     return file.getLastModified();
   }
 
-  /** Set the modified time of an existing file to now.
-   * @throws IOException if the file does not exist
-   */
-  @Override
-  public void touchFile(String name) throws IOException {
-    ensureOpen();
-    RAMFile file = fileMap.get(name);
-    if (file == null) {
-      throw new FileNotFoundException(name);
-    }
-    
-    long ts2, ts1 = System.currentTimeMillis();
-    do {
-      try {
-        Thread.sleep(0, 1);
-      } catch (InterruptedException ie) {
-        throw new ThreadInterruptedException(ie);
-      }
-      ts2 = System.currentTimeMillis();
-    } while(ts1 == ts2);
-    
-    file.setLastModified(ts2);
-  }
-
   /** Returns the length in bytes of a file in the directory.
    * @throws IOException if the file does not exist
    */

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/RAMFile.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/RAMFile.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/RAMFile.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/RAMFile.java Mon May 23 16:12:02 2011
@@ -26,7 +26,6 @@ public class RAMFile {
   RAMDirectory directory;
   protected long sizeInBytes;
 
-  // This is publicly modifiable via Directory.touchFile(), so direct access not supported
   private long lastModified = System.currentTimeMillis();
 
   // File used as buffer, in no RAMDirectory

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java Mon May 23 16:12:02 2011
@@ -125,7 +125,7 @@ public class SimpleFSDirectory extends F
           final OutOfMemoryError outOfMemoryError = new OutOfMemoryError(
               "OutOfMemoryError likely caused by the Sun VM Bug described in "
               + "https://issues.apache.org/jira/browse/LUCENE-1566; try calling FSDirectory.setReadChunkSize "
-              + "with a value smaller than the current chunks size (" + chunkSize + ")");
+              + "with a value smaller than the current chunk size (" + chunkSize + ")");
           outOfMemoryError.initCause(e);
           throw outOfMemoryError;
         }

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/util/StringHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/util/StringHelper.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/util/StringHelper.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/util/StringHelper.java Mon May 23 16:12:02 2011
@@ -1,5 +1,8 @@
 package org.apache.lucene.util;
 
+import java.util.Comparator;
+import java.util.StringTokenizer;
+
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -54,4 +57,42 @@ public abstract class StringHelper {
 
   private StringHelper() {
   }
+  
+  /**
+   * @return a Comparator over versioned strings such as X.YY.Z
+   * @lucene.internal
+   */
+  public static Comparator<String> getVersionComparator() {
+    return versionComparator;
+  }
+  
+  private static Comparator<String> versionComparator = new Comparator<String>() {
+    public int compare(String a, String b) {
+      StringTokenizer aTokens = new StringTokenizer(a, ".");
+      StringTokenizer bTokens = new StringTokenizer(b, ".");
+      
+      while (aTokens.hasMoreTokens()) {
+        int aToken = Integer.parseInt(aTokens.nextToken());
+        if (bTokens.hasMoreTokens()) {
+          int bToken = Integer.parseInt(bTokens.nextToken());
+          if (aToken != bToken) {
+            return aToken - bToken;
+          }
+        } else {
+          // a has some extra trailing tokens. if these are all zeroes, thats ok.
+          if (aToken != 0) {
+            return 1; 
+          }
+        }
+      }
+      
+      // b has some extra trailing tokens. if these are all zeroes, thats ok.
+      while (bTokens.hasMoreTokens()) {
+        if (Integer.parseInt(bTokens.nextToken()) != 0)
+          return -1;
+      }
+      
+      return 0;
+    }
+  };
 }

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/util/automaton/MinimizationOperations.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/util/automaton/MinimizationOperations.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/util/automaton/MinimizationOperations.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/util/automaton/MinimizationOperations.java Mon May 23 16:12:02 2011
@@ -30,6 +30,8 @@
 package org.apache.lucene.util.automaton;
 
 import java.util.BitSet;
+import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.LinkedList;
 
 /**
@@ -72,8 +74,12 @@ final public class MinimizationOperation
     final int[] sigma = a.getStartPoints();
     final State[] states = a.getNumberedStates();
     final int sigmaLen = sigma.length, statesLen = states.length;
-    final BitSet[][] reverse = new BitSet[statesLen][sigmaLen];
-    final BitSet[] splitblock = new BitSet[statesLen], partition = new BitSet[statesLen];
+    @SuppressWarnings("unchecked") final ArrayList<State>[][] reverse =
+      (ArrayList<State>[][]) new ArrayList[statesLen][sigmaLen];
+    @SuppressWarnings("unchecked") final HashSet<State>[] partition =
+      (HashSet<State>[]) new HashSet[statesLen];
+    @SuppressWarnings("unchecked") final ArrayList<State>[] splitblock =
+      (ArrayList<State>[]) new ArrayList[statesLen];
     final int[] block = new int[statesLen];
     final StateList[][] active = new StateList[statesLen][sigmaLen];
     final StateListNode[][] active2 = new StateListNode[statesLen][sigmaLen];
@@ -82,8 +88,8 @@ final public class MinimizationOperation
     final BitSet split = new BitSet(statesLen), 
       refine = new BitSet(statesLen), refine2 = new BitSet(statesLen);
     for (int q = 0; q < statesLen; q++) {
-      splitblock[q] = new BitSet(statesLen);
-      partition[q] = new BitSet(statesLen);
+      splitblock[q] = new ArrayList<State>();
+      partition[q] = new HashSet<State>();
       for (int x = 0; x < sigmaLen; x++) {
         active[q][x] = new StateList();
       }
@@ -92,23 +98,22 @@ final public class MinimizationOperation
     for (int q = 0; q < statesLen; q++) {
       final State qq = states[q];
       final int j = qq.accept ? 0 : 1;
-      partition[j].set(q);
+      partition[j].add(qq);
       block[q] = j;
       for (int x = 0; x < sigmaLen; x++) {
-        final BitSet[] r =
+        final ArrayList<State>[] r =
           reverse[qq.step(sigma[x]).number];
         if (r[x] == null)
-          r[x] = new BitSet();
-        r[x].set(q);
+          r[x] = new ArrayList<State>();
+        r[x].add(qq);
       }
     }
     // initialize active sets
     for (int j = 0; j <= 1; j++) {
-      final BitSet part = partition[j];
       for (int x = 0; x < sigmaLen; x++) {
-        for (int i = part.nextSetBit(0); i >= 0; i = part.nextSetBit(i+1)) {
-          if (reverse[i][x] != null)
-            active2[i][x] = active[j][x].add(states[i]);
+        for (final State qq : partition[j]) {
+          if (reverse[qq.number][x] != null)
+            active2[qq.number][x] = active[j][x].add(qq);
         }
       }
     }
@@ -121,18 +126,19 @@ final public class MinimizationOperation
     // process pending until fixed point
     int k = 2;
     while (!pending.isEmpty()) {
-      IntPair ip = pending.removeFirst();
+      final IntPair ip = pending.removeFirst();
       final int p = ip.n1;
       final int x = ip.n2;
       pending2.clear(x*statesLen + p);
       // find states that need to be split off their blocks
       for (StateListNode m = active[p][x].first; m != null; m = m.next) {
-        final BitSet r = reverse[m.q.number][x];
-        if (r != null) for (int i = r.nextSetBit(0); i >= 0; i = r.nextSetBit(i+1)) {
+        final ArrayList<State> r = reverse[m.q.number][x];
+        if (r != null) for (final State s : r) {
+          final int i = s.number;
           if (!split.get(i)) {
             split.set(i);
             final int j = block[i];
-            splitblock[j].set(i);
+            splitblock[j].add(s);
             if (!refine2.get(j)) {
               refine2.set(j);
               refine.set(j);
@@ -142,18 +148,19 @@ final public class MinimizationOperation
       }
       // refine blocks
       for (int j = refine.nextSetBit(0); j >= 0; j = refine.nextSetBit(j+1)) {
-        final BitSet sb = splitblock[j];
-        if (sb.cardinality() < partition[j].cardinality()) {
-          final BitSet b1 = partition[j], b2 = partition[k];
-          for (int i = sb.nextSetBit(0); i >= 0; i = sb.nextSetBit(i+1)) {
-            b1.clear(i);
-            b2.set(i);
-            block[i] = k;
+        final ArrayList<State> sb = splitblock[j];
+        if (sb.size() < partition[j].size()) {
+          final HashSet<State> b1 = partition[j];
+          final HashSet<State> b2 = partition[k];
+          for (final State s : sb) {
+            b1.remove(s);
+            b2.add(s);
+            block[s.number] = k;
             for (int c = 0; c < sigmaLen; c++) {
-              final StateListNode sn = active2[i][c];
+              final StateListNode sn = active2[s.number][c];
               if (sn != null && sn.sl == active[j][c]) {
                 sn.remove();
-                active2[i][c] = active[k][c].add(states[i]);
+                active2[s.number][c] = active[k][c].add(s);
               }
             }
           }
@@ -173,8 +180,8 @@ final public class MinimizationOperation
           k++;
         }
         refine2.clear(j);
-        for (int i = sb.nextSetBit(0); i >= 0; i = sb.nextSetBit(i+1))
-          split.clear(i);
+        for (final State s : sb)
+          split.clear(s.number);
         sb.clear();
       }
       refine.clear();
@@ -184,9 +191,7 @@ final public class MinimizationOperation
     for (int n = 0; n < newstates.length; n++) {
       final State s = new State();
       newstates[n] = s;
-      BitSet part = partition[n];
-      for (int i = part.nextSetBit(0); i >= 0; i = part.nextSetBit(i+1)) {
-        final State q = states[i];
+      for (State q : partition[n]) {
         if (q == a.initial) a.initial = s;
         s.accept = q.accept;
         s.number = q.number; // select representative

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java Mon May 23 16:12:02 2011
@@ -232,9 +232,7 @@ public class FST<T> {
 
   void setEmptyOutput(T v) throws IOException {
     if (emptyOutput != null) {
-      if (!emptyOutput.equals(v)) {
-        emptyOutput = outputs.merge(emptyOutput, v);
-      }
+      emptyOutput = outputs.merge(emptyOutput, v);
     } else {
       emptyOutput = v;
     }

Modified: lucene/dev/branches/flexscoring/lucene/src/site/src/documentation/content/xdocs/lucene-contrib/index.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/site/src/documentation/content/xdocs/lucene-contrib/index.xml?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/site/src/documentation/content/xdocs/lucene-contrib/index.xml (original)
+++ lucene/dev/branches/flexscoring/lucene/src/site/src/documentation/content/xdocs/lucene-contrib/index.xml Mon May 23 16:12:02 2011
@@ -50,11 +50,6 @@
                 <a href="http://svn.apache.org/repos/asf/lucene/dev/trunk/lucene/contrib/">http://svn.apache.org/repos/asf/lucene/dev/trunk/lucene/contrib/</a>.
             </p>
           
-            <section id="ant"><title>ant</title>
-                <p>Ant task to create Lucene indexes.</p>
-                <p>See <a href="../api/contrib-ant/index.html">ant javadoc</a></p>
-            </section>
-            
             <section id="benchmark"><title>benchmark</title>
                 <p>The benchmark contribution contains tools for benchmarking Lucene using standard, freely available corpora.</p>
                 <p>See <a href="../api/contrib-benchmark/index.html">benchmark javadoc</a></p>
@@ -66,11 +61,6 @@
                 <p>See <a href="../api/contrib-demo/index.html">demo javadoc</a></p>
             </section>
             
-            <section id="db"><title>db</title>
-                <p>Provides integration with Berkley DB.</p>
-                <p>See <a href="../api/contrib-db/index.html">db javadoc</a></p>
-            </section>
-            
             <section id="highlighter"><title>highlighter</title>
                 <p>A set of classes for highlighting matching terms in search results.</p>
                 <p>See <a href="../api/contrib-highlighter/index.html">highlighter javadoc</a></p>
@@ -80,11 +70,6 @@
                 <p>RAM-based index that enables much faster searching than RAMDirectory in certain situations.</p>
                 <p>See <a href="../api/contrib-instantiated/index.html">instantiated javadoc</a></p>
             </section>
-
-            <section id="lucli"><title>lucli</title>
-                <p>An application that allows Lucene index manipulation from the command-line.</p>
-                <p>See <a href="../api/contrib-lucli/index.html">lucli javadoc</a></p>
-            </section>
             
             <section id="memory"><title>memory</title>
                 <p>High-performance single-document main memory index.</p>
@@ -121,11 +106,6 @@
                 <p>See <a href="../api/contrib-spellchecker/index.html">spellchecker javadoc</a></p>
             </section>                     
             
-            <section id="swing"><title>swing</title>
-                <p>Swing components designed to integrate with Lucene.</p>
-                <p>See <a href="../api/contrib-swing/index.html">swing javadoc</a></p>
-            </section>    
-            
             <section id="wordnet"><title>wordnet</title>
                 <p>Tools to help utilize wordnet synonyms with Lucene</p>
                 <p>See <a href="../api/contrib-wordnet/index.html">wordnet javadoc</a></p>

Modified: lucene/dev/branches/flexscoring/lucene/src/site/src/documentation/content/xdocs/site.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/site/src/documentation/content/xdocs/site.xml?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/site/src/documentation/content/xdocs/site.xml (original)
+++ lucene/dev/branches/flexscoring/lucene/src/site/src/documentation/content/xdocs/site.xml Mon May 23 16:12:02 2011
@@ -52,14 +52,10 @@ See http://forrest.apache.org/docs/linki
 		 <javadoc-core label="Core" href="ext:javadocs-core"/>
 		 <javadoc-test-framework label="Test Framework" href="ext:javadocs-test-framework"/>
 		 <javadoc-contrib label="Contrib">
-		    <javadoc-contrib-ant label="Ant" href="ext:javadocs-contrib-ant"/>
-		    <javadoc-contrib-bdb label="Bdb" href="ext:javadocs-contrib-bdb"/>
-		    <javadoc-contrib-bdb-je label="Bdb-je" href="ext:javadocs-contrib-bdb-je"/>
 		    <javadoc-contrib-benchmark label="Benchmark" href="ext:javadocs-contrib-benchmark"/>
         <javadoc-contrib-demo label="Demo" href="ext:javadocs-contrib-demo"/>
 		    <javadoc-contrib-highlighter label="Highlighter" href="ext:javadocs-contrib-highlighter"/>
 		    <javadoc-contrib-instantiated label="Instantiated" href="ext:javadocs-contrib-instantiated"/>
-		    <javadoc-contrib-lucli label="Lucli" href="ext:javadocs-contrib-lucli"/>
 		    <javadoc-contrib-memory label="Memory" href="ext:javadocs-contrib-memory"/>
 		    <javadoc-contrib-misc label="Miscellaneous" href="ext:javadocs-contrib-misc"/>
 		    <javadoc-contrib-queries label="Queries" href="ext:javadocs-contrib-queries"/>
@@ -69,8 +65,7 @@ See http://forrest.apache.org/docs/linki
 -->			
 		    <javadoc-contrib-remote label="Remote" href="ext:javadocs-contrib-remote"/>
 		    <javadoc-contrib-spatial label="Spatial" href="ext:javadocs-contrib-spatial"/>
-		    <javadoc-contrib-spellchecker label="Spellchecker" href="ext:javadocs-contrib-spellchecker"/>			
-		    <javadoc-contrib-swing label="Swing" href="ext:javadocs-contrib-swing"/>				    
+		    <javadoc-contrib-spellchecker label="Spellchecker" href="ext:javadocs-contrib-spellchecker"/>						    
 			<javadoc-contrib-wordnet label="Wordnet" href="ext:javadocs-contrib-wordnet"/>			
 		    <javadoc-contrib-xml-query-parser label="XML Query Parser" href="ext:javadocs-contrib-xml-query-parser"/>			
 		 </javadoc-contrib>
@@ -100,14 +95,10 @@ See http://forrest.apache.org/docs/linki
     <javadocs-all href="api/all/index.html"/>
 	<javadocs-core href="api/core/index.html"/>
 	<javadocs-test-framework href="api/test-framework/index.html"/>
-	<javadocs-contrib-ant href="api/contrib-ant/index.html"/>
-	<javadocs-contrib-bdb href="api/contrib-bdb/index.html"/>
-	<javadocs-contrib-bdb-je href="api/contrib-bdb-je/index.html"/>
 	<javadocs-contrib-benchmark href="api/contrib-benchmark/index.html"/>
   <javadocs-contrib-demo href="api/contrib-demo/index.html"/>
 	<javadocs-contrib-highlighter href="api/contrib-highlighter/index.html"/>
 	<javadocs-contrib-instantiated href="api/contrib-instantiated/index.html"/>
-	<javadocs-contrib-lucli href="api/contrib-lucli/index.html"/>
 	<javadocs-contrib-memory href="api/contrib-memory/index.html"/>
 	<javadocs-contrib-misc href="api/contrib-misc/index.html"/>
 	<javadocs-contrib-queries href="api/contrib-queries/index.html"/>
@@ -115,7 +106,6 @@ See http://forrest.apache.org/docs/linki
 	<javadocs-contrib-remote href="api/contrib-remote/index.html"/>
 	<javadocs-contrib-spatial href="api/contrib-spatial/index.html"/>
 	<javadocs-contrib-spellchecker href="api/contrib-spellchecker/index.html"/>
-	<javadocs-contrib-swing href="api/contrib-swing/index.html"/>
 	<javadocs-contrib-wordnet href="api/contrib-wordnet/index.html"/>
 	<javadocs-contrib-xml-query-parser href="api/contrib-xml-query-parser/index.html"/>
 	

Modified: lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/analysis/BaseTokenStreamTestCase.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/analysis/BaseTokenStreamTestCase.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/analysis/BaseTokenStreamTestCase.java Mon May 23 16:12:02 2011
@@ -30,7 +30,15 @@ import org.apache.lucene.util.LuceneTest
 import org.apache.lucene.util._TestUtil;
 
 /** 
- * Base class for all Lucene unit tests that use TokenStreams.  
+ * Base class for all Lucene unit tests that use TokenStreams. 
+ * <p>
+ * When writing unit tests for analysis components, its highly recommended
+ * to use the helper methods here (especially in conjunction with {@link MockAnalyzer} or
+ * {@link MockTokenizer}), as they contain many assertions and checks to 
+ * catch bugs.
+ * 
+ * @see MockAnalyzer
+ * @see MockTokenizer
  */
 public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
   // some helpers to test Analyzers and TokenStreams:

Modified: lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/analysis/MockAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/analysis/MockAnalyzer.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/analysis/MockAnalyzer.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/analysis/MockAnalyzer.java Mon May 23 16:12:02 2011
@@ -27,6 +27,19 @@ import org.apache.lucene.util.automaton.
 
 /**
  * Analyzer for testing
+ * <p>
+ * This analyzer is a replacement for Whitespace/Simple/KeywordAnalyzers
+ * for unit tests. If you are testing a custom component such as a queryparser
+ * or analyzer-wrapper that consumes analysis streams, its a great idea to test
+ * it with this analyzer instead. MockAnalyzer has the following behavior:
+ * <ul>
+ *   <li>By default, the assertions in {@link MockTokenizer} are turned on for extra
+ *       checks that the consumer is consuming properly. These checks can be disabled
+ *       with {@link #setEnableChecks(boolean)}.
+ *   <li>Payload data is randomly injected into the stream for more thorough testing
+ *       of payloads.
+ * </ul>
+ * @see MockTokenizer
  */
 public final class MockAnalyzer extends Analyzer { 
   private final CharacterRunAutomaton runAutomaton;

Modified: lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/analysis/MockTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/analysis/MockTokenizer.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/analysis/MockTokenizer.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/analysis/MockTokenizer.java Mon May 23 16:12:02 2011
@@ -22,11 +22,21 @@ import java.io.Reader;
 
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.lucene.util.AttributeSource.AttributeFactory;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
 import org.apache.lucene.util.automaton.RegExp;
 
 /**
- * Automaton-based tokenizer for testing. Optionally lowercases.
+ * Tokenizer for testing.
+ * <p>
+ * This tokenizer is a replacement for {@link #WHITESPACE}, {@link #SIMPLE}, and {@link #KEYWORD}
+ * tokenizers. If you are writing a component such as a TokenFilter, its a great idea to test
+ * it wrapping this tokenizer instead for extra checks. This tokenizer has the following behavior:
+ * <ul>
+ *   <li>An internal state-machine is used for checking consumer consistency. These checks can
+ *       be disabled with {@link #setEnableChecks(boolean)}.
+ *   <li>For convenience, optionally lowercases terms that it outputs.
+ * </ul>
  */
 public class MockTokenizer extends Tokenizer {
   /** Acts Similar to WhitespaceTokenizer */
@@ -44,6 +54,8 @@ public class MockTokenizer extends Token
 
   private final CharacterRunAutomaton runAutomaton;
   private final boolean lowerCase;
+  private final int maxTokenLength;
+  public static final int DEFAULT_MAX_TOKEN_LENGTH = Integer.MAX_VALUE;
   private int state;
 
   private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
@@ -65,20 +77,21 @@ public class MockTokenizer extends Token
   private State streamState = State.CLOSE;
   private boolean enableChecks = true;
   
-  public MockTokenizer(AttributeFactory factory, Reader input, CharacterRunAutomaton runAutomaton, boolean lowerCase) {
+  public MockTokenizer(AttributeFactory factory, Reader input, CharacterRunAutomaton runAutomaton, boolean lowerCase, int maxTokenLength) {
     super(factory, input);
     this.runAutomaton = runAutomaton;
     this.lowerCase = lowerCase;
     this.state = runAutomaton.getInitialState();
     this.streamState = State.SETREADER;
+    this.maxTokenLength = maxTokenLength;
+  }
+
+  public MockTokenizer(Reader input, CharacterRunAutomaton runAutomaton, boolean lowerCase, int maxTokenLength) {
+    this(AttributeFactory.DEFAULT_ATTRIBUTE_FACTORY, input, runAutomaton, lowerCase, maxTokenLength);
   }
 
   public MockTokenizer(Reader input, CharacterRunAutomaton runAutomaton, boolean lowerCase) {
-    super(input);
-    this.runAutomaton = runAutomaton;
-    this.lowerCase = lowerCase;
-    this.state = runAutomaton.getInitialState();
-    this.streamState = State.SETREADER;
+    this(input, runAutomaton, lowerCase, DEFAULT_MAX_TOKEN_LENGTH);
   }
   
   @Override
@@ -98,9 +111,12 @@ public class MockTokenizer extends Token
           for (int i = 0; i < chars.length; i++)
             termAtt.append(chars[i]);
           endOffset = off;
+          if (termAtt.length() >= maxTokenLength) {
+            break;
+          }
           cp = readCodePoint();
         } while (cp >= 0 && isTokenChar(cp));
-        offsetAtt.setOffset(startOffset, endOffset);
+        offsetAtt.setOffset(correctOffset(startOffset), correctOffset(endOffset));
         streamState = State.INCREMENT;
         return true;
       }

Modified: lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/MockRandomMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/MockRandomMergePolicy.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/MockRandomMergePolicy.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/MockRandomMergePolicy.java Mon May 23 16:12:02 2011
@@ -42,14 +42,13 @@ public class MockRandomMergePolicy exten
 
     if (segmentInfos.size() > 1 && random.nextInt(5) == 3) {
       
-      SegmentInfos segmentInfos2 = new SegmentInfos();
-      segmentInfos2.addAll(segmentInfos);
-      Collections.shuffle(segmentInfos2, random);
+      List<SegmentInfo> segments = new ArrayList<SegmentInfo>(segmentInfos.asList());
+      Collections.shuffle(segments, random);
 
       // TODO: sometimes make more than 1 merge?
       mergeSpec = new MergeSpecification();
       final int segsToMerge = _TestUtil.nextInt(random, 1, segmentInfos.size());
-      mergeSpec.add(new OneMerge(segmentInfos2.range(0, segsToMerge)));
+      mergeSpec.add(new OneMerge(segments.subList(0, segsToMerge)));
     }
 
     return mergeSpec;

Modified: lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java Mon May 23 16:12:02 2011
@@ -196,7 +196,7 @@ public class MockRandomCodec extends Cod
 
               @Override
               public boolean isIndexTerm(BytesRef term, TermStats stats) {
-                return rand.nextInt(gap) == 17;
+                return rand.nextInt(gap) == gap/2;
               }
 
               @Override

Modified: lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java Mon May 23 16:12:02 2011
@@ -560,12 +560,6 @@ public class MockDirectoryWrapper extend
   }
 
   @Override
-  public synchronized void touchFile(String name) throws IOException {
-    maybeYield();
-    delegate.touchFile(name);
-  }
-
-  @Override
   public synchronized long fileLength(String name) throws IOException {
     maybeYield();
     return delegate.fileLength(name);

Modified: lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java Mon May 23 16:12:02 2011
@@ -66,6 +66,8 @@ import org.junit.runner.Description;
 import org.junit.runner.RunWith;
 import org.junit.runner.manipulation.Filter;
 import org.junit.runner.manipulation.NoTestsRemainException;
+import org.junit.runner.notification.Failure;
+import org.junit.runner.notification.RunListener;
 import org.junit.runner.notification.RunNotifier;
 import org.junit.runners.BlockJUnit4ClassRunner;
 import org.junit.runners.model.FrameworkMethod;
@@ -171,8 +173,15 @@ public abstract class LuceneTestCase ext
   private volatile Thread.UncaughtExceptionHandler savedUncaughtExceptionHandler = null;
 
   /** Used to track if setUp and tearDown are called correctly from subclasses */
-  private boolean setup;
+  private static State state = State.INITIAL;
 
+  private static enum State {
+    INITIAL, // no tests ran yet
+    SETUP,   // test has called setUp()
+    RANTEST, // test is running
+    TEARDOWN // test has called tearDown()
+  };
+  
   /**
    * Some tests expect the directory to contain a single segment, and want to do tests on that segment's reader.
    * This is an utility method to help them.
@@ -326,6 +335,7 @@ public abstract class LuceneTestCase ext
 
   @BeforeClass
   public static void beforeClassLuceneTestCaseJ4() {
+    state = State.INITIAL;
     staticSeed = "random".equals(TEST_SEED) ? seedRand.nextLong() : TwoLongs.fromString(TEST_SEED).l1;
     random.setSeed(staticSeed);
     tempDirs.clear();
@@ -375,6 +385,11 @@ public abstract class LuceneTestCase ext
 
   @AfterClass
   public static void afterClassLuceneTestCaseJ4() {
+    if (!testsFailed) {
+      assertTrue("ensure your setUp() calls super.setUp() and your tearDown() calls super.tearDown()!!!", 
+          state == State.INITIAL || state == State.TEARDOWN);
+    }
+    state = State.INITIAL;
     if (! "false".equals(TEST_CLEAN_THREADS)) {
       int rogueThreads = threadCleanup("test class");
       if (rogueThreads > 0) {
@@ -483,17 +498,22 @@ public abstract class LuceneTestCase ext
     public void starting(FrameworkMethod method) {
       // set current method name for logging
       LuceneTestCase.this.name = method.getName();
+      if (!testsFailed) {
+        assertTrue("ensure your setUp() calls super.setUp()!!!", state == State.SETUP);
+      }
+      state = State.RANTEST;
       super.starting(method);
     }
-
   };
 
   @Before
   public void setUp() throws Exception {
     seed = "random".equals(TEST_SEED) ? seedRand.nextLong() : TwoLongs.fromString(TEST_SEED).l2;
     random.setSeed(seed);
-    assertFalse("ensure your tearDown() calls super.tearDown()!!!", setup);
-    setup = true;
+    if (!testsFailed) {
+      assertTrue("ensure your tearDown() calls super.tearDown()!!!", (state == State.INITIAL || state == State.TEARDOWN));
+    }
+    state = State.SETUP;
     savedUncaughtExceptionHandler = Thread.getDefaultUncaughtExceptionHandler();
     Thread.setDefaultUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
       public void uncaughtException(Thread t, Throwable e) {
@@ -529,8 +549,12 @@ public abstract class LuceneTestCase ext
 
   @After
   public void tearDown() throws Exception {
-    assertTrue("ensure your setUp() calls super.setUp()!!!", setup);
-    setup = false;
+    if (!testsFailed) {
+      // Note: we allow a test to go straight from SETUP -> TEARDOWN (without ever entering the RANTEST state)
+      // because if you assume() inside setUp(), it skips the test and the TestWatchman has no way to know...
+      assertTrue("ensure your setUp() calls super.setUp()!!!", state == State.RANTEST || state == State.SETUP);
+    }
+    state = State.TEARDOWN;
     BooleanQuery.setMaxClauseCount(savedBoolMaxClauseCount);
     if ("perMethod".equals(TEST_CLEAN_THREADS)) {
       int rogueThreads = threadCleanup("test method: '" + getName() + "'");
@@ -1123,6 +1147,7 @@ public abstract class LuceneTestCase ext
    * with one that returns null for getSequentialSubReaders.
    */
   public static IndexSearcher newSearcher(IndexReader r, boolean maybeWrap) throws IOException {
+
     if (random.nextBoolean()) {
       if (maybeWrap && random.nextBoolean()) {
         return new IndexSearcher(new SlowMultiReaderWrapper(r));
@@ -1274,17 +1299,25 @@ public abstract class LuceneTestCase ext
       }
       
       // only print iteration info if the user requested more than one iterations
-      boolean verbose = VERBOSE && TEST_ITER > 1;
+      final boolean verbose = VERBOSE && TEST_ITER > 1;
+      
+      final int currentIter[] = new int[1];
+      arg1.addListener(new RunListener() {
+        @Override
+        public void testFailure(Failure failure) throws Exception {
+          if (verbose) {
+            System.out.println("\nNOTE: iteration " + currentIter[0] + " failed! ");
+          }
+        }
+      });
       for (int i = 0; i < TEST_ITER; i++) {
+        currentIter[0] = i;
         if (verbose) {
           System.out.println("\nNOTE: running iter=" + (1+i) + " of " + TEST_ITER);
         }
         super.runChild(arg0, arg1);
         if (testsFailed) {
-          if (i >= TEST_ITER_MIN - 1) {
-            if (verbose) {
-              System.out.println("\nNOTE: iteration " + i + " failed !");
-            }
+          if (i >= TEST_ITER_MIN - 1) { // XXX is this still off-by-one?
             break;
           }
         }

Modified: lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/analysis/TestToken.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/analysis/TestToken.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/analysis/TestToken.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/analysis/TestToken.java Mon May 23 16:12:02 2011
@@ -225,7 +225,7 @@ public class TestToken extends LuceneTes
   }
 
   public void testTokenAttributeFactory() throws Exception {
-    TokenStream ts = new MockTokenizer(Token.TOKEN_ATTRIBUTE_FACTORY, new StringReader("foo bar"), MockTokenizer.WHITESPACE, false);
+    TokenStream ts = new MockTokenizer(Token.TOKEN_ATTRIBUTE_FACTORY, new StringReader("foo bar"), MockTokenizer.WHITESPACE, false, MockTokenizer.DEFAULT_MAX_TOKEN_LENGTH);
     
     assertTrue("SenselessAttribute is not implemented by SenselessAttributeImpl",
       ts.addAttribute(SenselessAttribute.class) instanceof SenselessAttributeImpl);

Modified: lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestFieldsReader.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestFieldsReader.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestFieldsReader.java Mon May 23 16:12:02 2011
@@ -411,10 +411,6 @@ public class TestFieldsReader extends Lu
       return fsDir.fileModified(name);
     }
     @Override
-    public void touchFile(String name) throws IOException {
-      fsDir.touchFile(name);
-    }
-    @Override
     public void deleteFile(String name) throws IOException {
       fsDir.deleteFile(name);
     }

Modified: lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java Mon May 23 16:12:02 2011
@@ -30,7 +30,6 @@ import org.apache.lucene.store.LockObtai
 import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.LineFileDocs;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.ThrottledIndexOutput;
 import org.junit.Before;
 
 public class TestFlushByRamOrCountsPolicy extends LuceneTestCase {
@@ -105,7 +104,7 @@ public class TestFlushByRamOrCountsPolic
       assertTrue(maxRAMBytes < flushControl.peakActiveBytes);
     }
     if (ensureNotStalled) {
-      assertFalse(docsWriter.healthiness.wasStalled);
+      assertFalse(docsWriter.flushControl.stallControl.wasStalled);
     }
     writer.close();
     assertEquals(0, flushControl.activeBytes());
@@ -216,15 +215,15 @@ public class TestFlushByRamOrCountsPolic
     assertEquals(numDocumentsToIndex, r.numDocs());
     assertEquals(numDocumentsToIndex, r.maxDoc());
     if (!flushPolicy.flushOnRAM()) {
-      assertFalse("never stall if we don't flush on RAM", docsWriter.healthiness.wasStalled);
-      assertFalse("never block if we don't flush on RAM", docsWriter.healthiness.hasBlocked());
+      assertFalse("never stall if we don't flush on RAM", docsWriter.flushControl.stallControl.wasStalled);
+      assertFalse("never block if we don't flush on RAM", docsWriter.flushControl.stallControl.hasBlocked());
     }
     r.close();
     writer.close();
     dir.close();
   }
 
-  public void testHealthyness() throws InterruptedException,
+  public void testStallControl() throws InterruptedException,
       CorruptIndexException, LockObtainFailedException, IOException {
 
     int[] numThreads = new int[] { 4 + random.nextInt(8), 1 };
@@ -240,7 +239,7 @@ public class TestFlushByRamOrCountsPolic
       iwc.setMaxBufferedDeleteTerms(IndexWriterConfig.DISABLE_AUTO_FLUSH);
       FlushPolicy flushPolicy = new FlushByRamOrCountsPolicy();
       iwc.setFlushPolicy(flushPolicy);
-
+      
       DocumentsWriterPerThreadPool threadPool = new ThreadAffinityDocumentsWriterThreadPool(
           numThreads[i]== 1 ? 1 : 2);
       iwc.setIndexerThreadPool(threadPool);
@@ -265,11 +264,11 @@ public class TestFlushByRamOrCountsPolic
       assertEquals(numDocumentsToIndex, writer.maxDoc());
       if (numThreads[i] == 1) {
         assertFalse(
-            "single thread must not stall",
-            docsWriter.healthiness.wasStalled);
-        assertFalse(
             "single thread must not block numThreads: " + numThreads[i],
-            docsWriter.healthiness.hasBlocked());
+            docsWriter.flushControl.stallControl.hasBlocked());
+      }
+      if (docsWriter.flushControl.peakNetBytes > (2.d * iwc.getRAMBufferSizeMB() * 1024.d * 1024.d)) {
+        assertTrue(docsWriter.flushControl.stallControl.wasStalled);
       }
       assertActiveBytesAfter(flushControl);
       writer.close(true);

Modified: lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestGlobalFieldNumbers.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestGlobalFieldNumbers.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestGlobalFieldNumbers.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestGlobalFieldNumbers.java Mon May 23 16:12:02 2011
@@ -363,7 +363,7 @@ public class TestGlobalFieldNumbers exte
       w.close();
       SegmentInfos sis = new SegmentInfos();
       sis.read(base);
-      SegmentInfo segmentInfo = sis.get(sis.size() - 1);// last segment must
+      SegmentInfo segmentInfo = sis.info(sis.size() - 1);// last segment must
                                                         // have all fields with
                                                         // consistent numbers
       FieldInfos fieldInfos = segmentInfo.getFieldInfos();

Modified: lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestIndexReaderClone.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestIndexReaderClone.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestIndexReaderClone.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestIndexReaderClone.java Mon May 23 16:12:02 2011
@@ -17,7 +17,7 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.SegmentReader.Norm;
+import org.apache.lucene.index.SegmentNorms;
 import org.apache.lucene.search.DefaultSimilarity;
 import org.apache.lucene.search.Similarity;
 import org.apache.lucene.analysis.MockAnalyzer;
@@ -338,7 +338,7 @@ public class TestIndexReaderClone extend
     origSegmentReader.close();
     assertDelDocsRefCountEquals(1, origSegmentReader);
     // check the norm refs
-    Norm norm = clonedSegmentReader.norms.get("field1");
+    SegmentNorms norm = clonedSegmentReader.norms.get("field1");
     assertEquals(1, norm.bytesRef().get());
     clonedSegmentReader.close();
     dir1.close();

Modified: lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestIndexReaderCloneNorms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestIndexReaderCloneNorms.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestIndexReaderCloneNorms.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestIndexReaderCloneNorms.java Mon May 23 16:12:02 2011
@@ -29,7 +29,7 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.Field.Index;
 import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
-import org.apache.lucene.index.SegmentReader.Norm;
+import org.apache.lucene.index.SegmentNorms;
 import org.apache.lucene.search.DefaultSimilarity;
 import org.apache.lucene.search.DefaultSimilarityProvider;
 import org.apache.lucene.search.Similarity;
@@ -184,7 +184,7 @@ public class TestIndexReaderCloneNorms e
     TestIndexReaderReopen.createIndex(random, dir1, false);
     SegmentReader reader1 = getOnlySegmentReader(IndexReader.open(dir1, false));
     reader1.norms("field1");
-    Norm r1norm = reader1.norms.get("field1");
+    SegmentNorms r1norm = reader1.norms.get("field1");
     AtomicInteger r1BytesRef = r1norm.bytesRef();
     SegmentReader reader2 = (SegmentReader)reader1.clone();
     assertEquals(2, r1norm.bytesRef().get());
@@ -203,14 +203,14 @@ public class TestIndexReaderCloneNorms e
     IndexReader reader2C = (IndexReader) reader1.clone();
     SegmentReader segmentReader2C = getOnlySegmentReader(reader2C);
     segmentReader2C.norms("field1"); // load the norms for the field
-    Norm reader2CNorm = segmentReader2C.norms.get("field1");
+    SegmentNorms reader2CNorm = segmentReader2C.norms.get("field1");
     assertTrue("reader2CNorm.bytesRef()=" + reader2CNorm.bytesRef(), reader2CNorm.bytesRef().get() == 2);
     
     
     
     IndexReader reader3C = (IndexReader) reader2C.clone();
     SegmentReader segmentReader3C = getOnlySegmentReader(reader3C);
-    Norm reader3CCNorm = segmentReader3C.norms.get("field1");
+    SegmentNorms reader3CCNorm = segmentReader3C.norms.get("field1");
     assertEquals(3, reader3CCNorm.bytesRef().get());
     
     // edit a norm and the refcount should be 1
@@ -231,13 +231,13 @@ public class TestIndexReaderCloneNorms e
     // norm values should be different 
     assertTrue(sim.decodeNormValue(segmentReader3C.norms("field1")[5]) 
     		!= sim.decodeNormValue(segmentReader4C.norms("field1")[5]));
-    Norm reader4CCNorm = segmentReader4C.norms.get("field1");
+    SegmentNorms reader4CCNorm = segmentReader4C.norms.get("field1");
     assertEquals(3, reader3CCNorm.bytesRef().get());
     assertEquals(1, reader4CCNorm.bytesRef().get());
         
     IndexReader reader5C = (IndexReader) reader4C.clone();
     SegmentReader segmentReader5C = getOnlySegmentReader(reader5C);
-    Norm reader5CCNorm = segmentReader5C.norms.get("field1");
+    SegmentNorms reader5CCNorm = segmentReader5C.norms.get("field1");
     reader5C.setNorm(5, "field1", sim.encodeNormValue(0.7f));
     assertEquals(1, reader5CCNorm.bytesRef().get());
 

Modified: lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java Mon May 23 16:12:02 2011
@@ -1045,11 +1045,11 @@ public class TestIndexWriter extends Luc
             newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).
                 setOpenMode(OpenMode.CREATE).
                 setMaxBufferedDocs(2).
-                setMergePolicy(newLogMergePolicy(101))
+                setMergePolicy(newLogMergePolicy(51))
         );
         Document doc = new Document();
         doc.add(newField("field", "aaa", Store.YES, Index.ANALYZED, TermVector.WITH_POSITIONS_OFFSETS));
-        for(int i=0;i<200;i++)
+        for(int i=0;i<100;i++)
           writer.addDocument(doc);
         writer.optimize(false);
 
@@ -1231,13 +1231,17 @@ public class TestIndexWriter extends Luc
         System.out.println("TEST: pass=" + pass);
       }
 
-      IndexWriter writer = new IndexWriter(
-          directory,
-          newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).
+      IndexWriterConfig conf =  newIndexWriterConfig(
+              TEST_VERSION_CURRENT, new MockAnalyzer(random)).
               setOpenMode(OpenMode.CREATE).
               setMaxBufferedDocs(2).
-              setMergePolicy(newLogMergePolicy())
-      );
+              setMergePolicy(newLogMergePolicy());
+      if (pass == 2) {
+        conf.setMergeScheduler(new SerialMergeScheduler());
+      }
+
+      IndexWriter writer = new IndexWriter(directory, conf);
+      ((LogMergePolicy) writer.getConfig().getMergePolicy()).setMergeFactor(100);          
       writer.setInfoStream(VERBOSE ? System.out : null);
 
       for(int iter=0;iter<10;iter++) {
@@ -2139,7 +2143,7 @@ public class TestIndexWriter extends Luc
       while(!finish) {
         try {
 
-          while(true) {
+          while(!finish) {
             if (w != null) {
               w.close();
               w = null;
@@ -2157,6 +2161,7 @@ public class TestIndexWriter extends Luc
               }
             }
             w.close();
+            w = null;
             _TestUtil.checkIndex(dir);
             IndexReader.open(dir, true).close();
 

Modified: lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestIndexWriterConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestIndexWriterConfig.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestIndexWriterConfig.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestIndexWriterConfig.java Mon May 23 16:12:02 2011
@@ -71,9 +71,6 @@ public class TestIndexWriterConfig exten
     assertEquals(ThreadAffinityDocumentsWriterThreadPool.class, conf.getIndexerThreadPool().getClass());
     assertNull(conf.getFlushPolicy());
     assertEquals(IndexWriterConfig.DEFAULT_RAM_PER_THREAD_HARD_LIMIT_MB, conf.getRAMPerThreadHardLimitMB());
-
-
-
     // Sanity check - validate that all getters are covered.
     Set<String> getters = new HashSet<String>();
     getters.add("getAnalyzer");

Modified: lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestNRTThreads.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestNRTThreads.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestNRTThreads.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestNRTThreads.java Mon May 23 16:12:02 2011
@@ -71,6 +71,21 @@ public class TestNRTThreads extends Luce
     final File tempDir = _TestUtil.getTempDir("nrtopenfiles");
     final MockDirectoryWrapper dir = new MockDirectoryWrapper(random, FSDirectory.open(tempDir));
     final IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random));
+
+    if (LuceneTestCase.TEST_NIGHTLY) {
+      // newIWConfig makes smallish max seg size, which
+      // results in tons and tons of segments for this test
+      // when run nightly:
+      MergePolicy mp = conf.getMergePolicy();
+      if (mp instanceof TieredMergePolicy) {
+        ((TieredMergePolicy) mp).setMaxMergedSegmentMB(5000.);
+      } else if (mp instanceof LogByteSizeMergePolicy) {
+        ((LogByteSizeMergePolicy) mp).setMaxMergeMB(1000.);
+      } else if (mp instanceof LogMergePolicy) {
+        ((LogMergePolicy) mp).setMaxMergeDocs(100000);
+      }
+    }
+
     conf.setMergedSegmentWarmer(new IndexWriter.IndexReaderWarmer() {
       @Override
       public void warm(IndexReader reader) throws IOException {

Modified: lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java Mon May 23 16:12:02 2011
@@ -128,8 +128,8 @@ public class TestPerSegmentDeletes exten
     fsmp.length = 2;
     System.out.println("maybeMerge "+writer.segmentInfos);
 
-    SegmentInfo info0 = writer.segmentInfos.get(0);
-    SegmentInfo info1 = writer.segmentInfos.get(1);
+    SegmentInfo info0 = writer.segmentInfos.info(0);
+    SegmentInfo info1 = writer.segmentInfos.info(1);
 
     writer.maybeMerge();
     System.out.println("maybeMerge after "+writer.segmentInfos);
@@ -199,7 +199,7 @@ public class TestPerSegmentDeletes exten
     // deletes for info1, the newly created segment from the
     // merge should have no deletes because they were applied in
     // the merge
-    //SegmentInfo info1 = writer.segmentInfos.get(1);
+    //SegmentInfo info1 = writer.segmentInfos.info(1);
     //assertFalse(exists(info1, writer.docWriter.segmentDeletes));
 
     //System.out.println("infos4:"+writer.segmentInfos);
@@ -261,11 +261,7 @@ public class TestPerSegmentDeletes exten
         throws CorruptIndexException, IOException {
       MergeSpecification ms = new MergeSpecification();
       if (doMerge) {
-        SegmentInfos mergeInfos = new SegmentInfos();
-        for (int x=start; x < (start+length); x++) {
-          mergeInfos.add(segmentInfos.get(x));
-        }
-        OneMerge om = new OneMerge(mergeInfos);
+        OneMerge om = new OneMerge(segmentInfos.asList().subList(start, start + length));
         ms.add(om);
         doMerge = false;
         return ms;

Modified: lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/index.32.cfs.zip
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/index.32.cfs.zip?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
Binary files - no diff available.

Modified: lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/index.32.nocfs.zip
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/index.32.nocfs.zip?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
Binary files - no diff available.