You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2010/10/27 17:16:57 UTC

svn commit: r1027998 - in /lucene/dev/trunk/lucene: ./ contrib/highlighter/src/java/org/apache/lucene/search/highlight/ contrib/instantiated/src/java/org/apache/lucene/store/instantiated/ contrib/memory/src/java/org/apache/lucene/index/memory/ contrib/...

Author: uschindler
Date: Wed Oct 27 15:16:56 2010
New Revision: 1027998

URL: http://svn.apache.org/viewvc?rev=1027998&view=rev
Log:
LUCENE-2719: Readded utility class org.apche.lucene.util.SorterTemplate with support for faster quickSort using pivot values and also merge sort and insertion sort. This removes code duplication and slow Collections/Arrays.sort usage.

Added:
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/CollectionUtil.java   (with props)
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/SorterTemplate.java   (with props)
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestCollectionUtil.java   (with props)
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/NOTICE.txt
    lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/TokenSources.java
    lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/TokenStreamFromTermPositionVector.java
    lucene/dev/trunk/lucene/contrib/instantiated/src/java/org/apache/lucene/store/instantiated/InstantiatedIndexWriter.java
    lucene/dev/trunk/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
    lucene/dev/trunk/lucene/contrib/spellchecker/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexReader.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/ConjunctionScorer.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PhraseQuery.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/QueryTermVector.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/ArrayUtil.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/BytesRefHash.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/BasicOperations.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/State.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestArrayUtil.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1027998&r1=1027997&r2=1027998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Wed Oct 27 15:16:56 2010
@@ -324,9 +324,11 @@ Bug fixes
 
 Changes in backwards compatibility policy
 
-* LUCENE-1483: Removed utility class oal.util.SorterTemplate; this
-  class is no longer used by Lucene.  (Gunnar Wagenknecht via Mike
-  McCandless)
+* LUCENE-2719: Changed API of internal utility class
+  org.apche.lucene.util.SorterTemplate to support faster quickSort using
+  pivot values and also merge sort and insertion sort. If you have used
+  this class, you have to implement two more methods for handling pivots.
+  (Uwe Schindler, Robert Muir, Mike McCandless)
 
 * LUCENE-1923: Renamed SegmentInfo & SegmentInfos segString method to
   toString.  These are advanced APIs and subject to change suddenly.

Modified: lucene/dev/trunk/lucene/NOTICE.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/NOTICE.txt?rev=1027998&r1=1027997&r2=1027998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/NOTICE.txt (original)
+++ lucene/dev/trunk/lucene/NOTICE.txt Wed Oct 27 15:16:56 2010
@@ -22,3 +22,8 @@ automatically generated with the moman/f
 Jean-Philippe Barrette-LaPierre. This library is available under an MIT license,
 see http://sites.google.com/site/rrettesite/moman and 
 http://bitbucket.org/jpbarrette/moman/overview/
+
+The class org.apache.lucene.SorterTemplate was inspired by CGLIB's class with
+the same name. The implementation part is mainly done using pre-existing
+Lucene sorting code. In-place stable mergesort was borrowed from CGLIB,
+which is Apache-licensed.

Modified: lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/TokenSources.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/TokenSources.java?rev=1027998&r1=1027997&r2=1027998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/TokenSources.java (original)
+++ lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/TokenSources.java Wed Oct 27 15:16:56 2010
@@ -23,7 +23,6 @@ package org.apache.lucene.search.highlig
 import java.io.IOException;
 import java.io.StringReader;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Comparator;
 
 import org.apache.lucene.analysis.Analyzer;
@@ -36,6 +35,7 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.TermFreqVector;
 import org.apache.lucene.index.TermPositionVector;
 import org.apache.lucene.index.TermVectorOffsetInfo;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 
 /**
@@ -231,13 +231,9 @@ public class TokenSources {
     if (unsortedTokens != null) {
       tokensInOriginalOrder = unsortedTokens.toArray(new Token[unsortedTokens
           .size()]);
-      Arrays.sort(tokensInOriginalOrder, new Comparator<Token>() {
+      ArrayUtil.quickSort(tokensInOriginalOrder, new Comparator<Token>() {
         public int compare(Token t1, Token t2) {
-          if (t1.startOffset() > t2.endOffset())
-            return 1;
-          if (t1.startOffset() < t2.startOffset())
-            return -1;
-          return 0;
+          return t1.startOffset() - t2.endOffset();
         }
       });
     }

Modified: lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/TokenStreamFromTermPositionVector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/TokenStreamFromTermPositionVector.java?rev=1027998&r1=1027997&r2=1027998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/TokenStreamFromTermPositionVector.java (original)
+++ lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/TokenStreamFromTermPositionVector.java Wed Oct 27 15:16:56 2010
@@ -18,7 +18,6 @@ package org.apache.lucene.search.highlig
  */
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.Comparator;
 import java.util.Iterator;
 import java.util.List;
@@ -31,6 +30,7 @@ import org.apache.lucene.analysis.tokena
 import org.apache.lucene.index.TermPositionVector;
 import org.apache.lucene.index.TermVectorOffsetInfo;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CollectionUtil;
 
 public final class TokenStreamFromTermPositionVector extends TokenStream {
 
@@ -75,18 +75,7 @@ public final class TokenStreamFromTermPo
         this.positionedTokens.add(token);
       }
     }
-    final Comparator<Token> tokenComparator = new Comparator<Token>() {
-      public int compare(final Token o1, final Token o2) {
-        if (o1.getPositionIncrement() < o2.getPositionIncrement()) {
-          return -1;
-        }
-        if (o1.getPositionIncrement() > o2.getPositionIncrement()) {
-          return 1;
-        }
-        return 0;
-      }
-    };
-    Collections.sort(this.positionedTokens, tokenComparator);
+    CollectionUtil.mergeSort(this.positionedTokens, tokenComparator);
     int lastPosition = -1;
     for (final Token token : this.positionedTokens) {
       int thisPosition = token.getPositionIncrement();
@@ -96,6 +85,12 @@ public final class TokenStreamFromTermPo
     this.tokensAtCurrentPosition = this.positionedTokens.iterator();
   }
 
+  private static final Comparator<Token> tokenComparator = new Comparator<Token>() {
+    public int compare(final Token o1, final Token o2) {
+      return o1.getPositionIncrement() - o2.getPositionIncrement();
+    }
+  };
+  
   @Override
   public boolean incrementToken() throws IOException {
     if (this.tokensAtCurrentPosition.hasNext()) {

Modified: lucene/dev/trunk/lucene/contrib/instantiated/src/java/org/apache/lucene/store/instantiated/InstantiatedIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/instantiated/src/java/org/apache/lucene/store/instantiated/InstantiatedIndexWriter.java?rev=1027998&r1=1027997&r2=1027998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/instantiated/src/java/org/apache/lucene/store/instantiated/InstantiatedIndexWriter.java (original)
+++ lucene/dev/trunk/lucene/contrib/instantiated/src/java/org/apache/lucene/store/instantiated/InstantiatedIndexWriter.java Wed Oct 27 15:16:56 2010
@@ -44,6 +44,8 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermVectorOffsetInfo;
 import org.apache.lucene.search.Similarity;
 import org.apache.lucene.util.StringHelper;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.CollectionUtil;
 import org.apache.lucene.util.AttributeImpl;
 import org.apache.lucene.util.BitVector;
 
@@ -345,11 +347,7 @@ public class InstantiatedIndexWriter imp
 
       for (Map.Entry<String, List<InstantiatedTermDocumentInformation>> eField_TermDocInfos : termDocumentInformationsByField.entrySet()) {
 
-        Collections.sort(eField_TermDocInfos.getValue(), new Comparator<InstantiatedTermDocumentInformation>() {
-          public int compare(InstantiatedTermDocumentInformation instantiatedTermDocumentInformation, InstantiatedTermDocumentInformation instantiatedTermDocumentInformation1) {
-            return instantiatedTermDocumentInformation.getTerm().getTerm().compareTo(instantiatedTermDocumentInformation1.getTerm().getTerm());
-          }
-        });
+        CollectionUtil.quickSort(eField_TermDocInfos.getValue(), tdComp);
 
         // add term vector
         if (documentFieldSettingsByFieldName.get(eField_TermDocInfos.getKey()).storeTermVector) {
@@ -366,7 +364,7 @@ public class InstantiatedIndexWriter imp
     // order document informations in dirty terms
     for (InstantiatedTerm term : dirtyTerms) {
       // todo optimize, i believe this is useless, that the natural order is document number?
-      Arrays.sort(term.getAssociatedDocuments(), InstantiatedTermDocumentInformation.documentNumberComparator);
+      ArrayUtil.mergeSort(term.getAssociatedDocuments(), InstantiatedTermDocumentInformation.documentNumberComparator);
 
 //      // update association class reference for speedy skipTo()
 //      for (int i = 0; i < term.getAssociatedDocuments().length; i++) {
@@ -426,6 +424,12 @@ public class InstantiatedIndexWriter imp
 
   }
 
+  private static final Comparator<InstantiatedTermDocumentInformation> tdComp = new Comparator<InstantiatedTermDocumentInformation>() {
+    public int compare(InstantiatedTermDocumentInformation instantiatedTermDocumentInformation, InstantiatedTermDocumentInformation instantiatedTermDocumentInformation1) {
+      return instantiatedTermDocumentInformation.getTerm().getTerm().compareTo(instantiatedTermDocumentInformation1.getTerm().getTerm());
+    }
+  };
+
   /**
    * Adds a document to this index.  If the document contains more than
    * {@link #setMaxFieldLength(int)} terms for a given field, the remainder are

Modified: lucene/dev/trunk/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java?rev=1027998&r1=1027997&r2=1027998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java (original)
+++ lucene/dev/trunk/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java Wed Oct 27 15:16:56 2010
@@ -55,6 +55,7 @@ import org.apache.lucene.search.Searcher
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Similarity;
 import org.apache.lucene.store.RAMDirectory; // for javadocs
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.Constants; // for javadocs
@@ -522,7 +523,7 @@ public class MemoryIndex implements Seri
       entries[i] = iter.next();
     }
     
-    if (size > 1) Arrays.sort(entries, termComparator);
+    if (size > 1) ArrayUtil.quickSort(entries, termComparator);
     return entries;
   }
   

Modified: lucene/dev/trunk/lucene/contrib/spellchecker/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/spellchecker/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java?rev=1027998&r1=1027997&r2=1027998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/spellchecker/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java (original)
+++ lucene/dev/trunk/lucene/contrib/spellchecker/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java Wed Oct 27 15:16:56 2010
@@ -18,7 +18,6 @@ package org.apache.lucene.search.spell;
  */
 
 import java.io.IOException;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
@@ -30,6 +29,7 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.FuzzyTermsEnum;
 import org.apache.lucene.search.MultiTermQuery;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.automaton.LevenshteinAutomata;
@@ -376,7 +376,7 @@ public class DirectSpellChecker {
       suggestions[index--] = suggestion;
     }
     
-    Arrays.sort(suggestions, Collections.reverseOrder(comparator));
+    ArrayUtil.mergeSort(suggestions, Collections.reverseOrder(comparator));
     if (numSug < suggestions.length) {
       SuggestWord trimmed[] = new SuggestWord[numSug];
       System.arraycopy(suggestions, 0, trimmed, 0, numSug);

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java?rev=1027998&r1=1027997&r2=1027998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java Wed Oct 27 15:16:56 2010
@@ -19,12 +19,12 @@ package org.apache.lucene.index;
 
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.ThreadInterruptedException;
+import org.apache.lucene.util.CollectionUtil;
 
 import java.io.IOException;
 import java.util.List;
 import java.util.ArrayList;
 import java.util.Comparator;
-import java.util.Collections;
 
 /** A {@link MergeScheduler} that runs each merge using a
  *  separate thread.
@@ -138,7 +138,7 @@ public class ConcurrentMergeScheduler ex
   }
 
   // Larger merges come first
-  protected static class CompareByMergeDocCount implements Comparator<MergeThread> {
+  protected static final Comparator<MergeThread> compareByMergeDocCount = new Comparator<MergeThread>() {
     public int compare(MergeThread t1, MergeThread t2) {
       final MergePolicy.OneMerge m1 = t1.getCurrentMerge();
       final MergePolicy.OneMerge m2 = t2.getCurrentMerge();
@@ -148,13 +148,13 @@ public class ConcurrentMergeScheduler ex
 
       return c2 - c1;
     }
-  }
+  };
 
   /** Called whenever the running merges have changed, to
    *  pause & unpause threads. */
   protected synchronized void updateMergeThreads() {
 
-    Collections.sort(mergeThreads, new CompareByMergeDocCount());
+    CollectionUtil.mergeSort(mergeThreads, compareByMergeDocCount);
     
     final int count = mergeThreads.size();
     int pri = mergeThreadPriority;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java?rev=1027998&r1=1027997&r2=1027998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java Wed Oct 27 15:16:56 2010
@@ -17,6 +17,7 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
+import java.util.Comparator;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
@@ -242,7 +243,7 @@ final class DocFieldProcessorPerThread e
     // sort the subset of fields that have vectors
     // enabled; we could save [small amount of] CPU
     // here.
-    quickSort(fields, 0, fieldCount-1);
+    ArrayUtil.quickSort(fields, 0, fieldCount, fieldsComp);
 
     for(int i=0;i<fieldCount;i++)
       fields[i].consumer.processFields(fields[i].fields, fields[i].fieldCount);
@@ -268,67 +269,12 @@ final class DocFieldProcessorPerThread e
       return both;
     }
   }
-
-  void quickSort(DocFieldProcessorPerField[] array, int lo, int hi) {
-    if (lo >= hi)
-      return;
-    else if (hi == 1+lo) {
-      if (array[lo].fieldInfo.name.compareTo(array[hi].fieldInfo.name) > 0) {
-        final DocFieldProcessorPerField tmp = array[lo];
-        array[lo] = array[hi];
-        array[hi] = tmp;
-      }
-      return;
-    }
-
-    int mid = (lo + hi) >>> 1;
-
-    if (array[lo].fieldInfo.name.compareTo(array[mid].fieldInfo.name) > 0) {
-      DocFieldProcessorPerField tmp = array[lo];
-      array[lo] = array[mid];
-      array[mid] = tmp;
-    }
-
-    if (array[mid].fieldInfo.name.compareTo(array[hi].fieldInfo.name) > 0) {
-      DocFieldProcessorPerField tmp = array[mid];
-      array[mid] = array[hi];
-      array[hi] = tmp;
-
-      if (array[lo].fieldInfo.name.compareTo(array[mid].fieldInfo.name) > 0) {
-        DocFieldProcessorPerField tmp2 = array[lo];
-        array[lo] = array[mid];
-        array[mid] = tmp2;
-      }
+  
+  private static final Comparator<DocFieldProcessorPerField> fieldsComp = new Comparator<DocFieldProcessorPerField>() {
+    public int compare(DocFieldProcessorPerField o1, DocFieldProcessorPerField o2) {
+      return o1.fieldInfo.name.compareTo(o2.fieldInfo.name);
     }
-
-    int left = lo + 1;
-    int right = hi - 1;
-
-    if (left >= right)
-      return;
-
-    DocFieldProcessorPerField partition = array[mid];
-
-    for (; ;) {
-      while (array[right].fieldInfo.name.compareTo(partition.fieldInfo.name) > 0)
-        --right;
-
-      while (left < right && array[left].fieldInfo.name.compareTo(partition.fieldInfo.name) <= 0)
-        ++left;
-
-      if (left < right) {
-        DocFieldProcessorPerField tmp = array[left];
-        array[left] = array[right];
-        array[right] = tmp;
-        --right;
-      } else {
-        break;
-      }
-    }
-
-    quickSort(array, lo, left);
-    quickSort(array, left + 1, hi);
-  }
+  };
 
   PerDoc[] docFreeList = new PerDoc[1];
   int freeCount;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java?rev=1027998&r1=1027997&r2=1027998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java Wed Oct 27 15:16:56 2010
@@ -20,7 +20,6 @@ package org.apache.lucene.index;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -30,6 +29,7 @@ import org.apache.lucene.index.codecs.Po
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.TermsConsumer;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CollectionUtil;
 
 final class FreqProxTermsWriter extends TermsHashConsumer {
 
@@ -75,7 +75,7 @@ final class FreqProxTermsWriter extends 
     final int numAllFields = allFields.size();
 
     // Sort by field name
-    Collections.sort(allFields);
+    CollectionUtil.quickSort(allFields);
 
     // TODO: allow Lucene user to customize this codec:
     final FieldsConsumer consumer = state.codec.fieldsConsumer(state);

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=1027998&r1=1027997&r2=1027998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java Wed Oct 27 15:16:56 2010
@@ -32,6 +32,7 @@ import java.util.Map;
 import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.NoSuchDirectoryException;
+import org.apache.lucene.util.CollectionUtil;
 
 /*
  * This class keeps track of each SegmentInfos instance that
@@ -235,7 +236,7 @@ final class IndexFileDeleter {
     }
 
     // We keep commits list in sorted order (oldest to newest):
-    Collections.sort(commits);
+    CollectionUtil.mergeSort(commits);
 
     // Now delete anything with ref count at 0.  These are
     // presumably abandoned files eg due to crash of

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexReader.java?rev=1027998&r1=1027997&r2=1027998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexReader.java Wed Oct 27 15:16:56 2010
@@ -22,6 +22,7 @@ import org.apache.lucene.document.FieldS
 import org.apache.lucene.search.Similarity;
 import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.store.*;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.ReaderUtil;         // for javadocs
@@ -1250,7 +1251,7 @@ public abstract class IndexReader implem
       cfr = new CompoundFileReader(dir, filename);
 
       String [] files = cfr.listAll();
-      Arrays.sort(files);   // sort the array of filename so that the output is more readable
+      ArrayUtil.quickSort(files);   // sort the array of filename so that the output is more readable
 
       for (int i = 0; i < files.length; ++i) {
         long len = cfr.fileLength(files[i]);

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/ConjunctionScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/ConjunctionScorer.java?rev=1027998&r1=1027997&r2=1027998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/ConjunctionScorer.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/ConjunctionScorer.java Wed Oct 27 15:16:56 2010
@@ -17,9 +17,9 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
+import org.apache.lucene.util.ArrayUtil;
 import java.io.IOException;
 import java.util.Collection;
-import java.util.Arrays;
 import java.util.Comparator;
 
 /** Scorer for conjunctions, sets of queries, all of which are required. */
@@ -51,8 +51,10 @@ class ConjunctionScorer extends Scorer {
     // We don't need to sort the array in any future calls because we know
     // it will already start off sorted (all scorers on same doc).
     
-    // note that this comparator is not consistent with equals!
-    Arrays.sort(scorers, new Comparator<Scorer>() {         // sort the array
+    // Note that this comparator is not consistent with equals!
+    // Also we use mergeSort here to be stable (so order of Scoreres that
+    // match on first document keeps preserved):
+    ArrayUtil.mergeSort(scorers, new Comparator<Scorer>() { // sort the array
       public int compare(Scorer o1, Scorer o2) {
         return o1.docID() - o2.docID();
       }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java?rev=1027998&r1=1027997&r2=1027998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java Wed Oct 27 15:16:56 2010
@@ -24,6 +24,7 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.util.PriorityQueue;
@@ -213,7 +214,7 @@ public class MultiPhraseQuery extends Qu
 
       // sort by increasing docFreq order
       if (slop == 0) {
-        Arrays.sort(postingsFreqs);
+        ArrayUtil.quickSort(postingsFreqs);
       }
 
       if (slop == 0) {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java?rev=1027998&r1=1027997&r2=1027998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java Wed Oct 27 15:16:56 2010
@@ -19,7 +19,6 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 import java.io.Serializable;
-import java.util.Arrays;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -507,13 +506,7 @@ public abstract class MultiTermQuery ext
       final Term placeholderTerm = new Term(query.field);
       final BooleanQuery bq = new BooleanQuery(true);
       final ScoreTerm[] scoreTerms = stQueue.toArray(new ScoreTerm[stQueue.size()]);
-      Arrays.sort(scoreTerms, new Comparator<ScoreTerm>() {
-        public int compare(ScoreTerm st1, ScoreTerm st2) {
-          assert st1.termComp == st2.termComp :
-            "term comparator should not change between segments";
-          return st1.termComp.compare(st1.bytes, st2.bytes);
-        }
-      });
+      ArrayUtil.quickSort(scoreTerms, scoreTermSortByTermComp);
       for (final ScoreTerm st : scoreTerms) {
         final Term term = placeholderTerm.createTerm(st.bytes);
         assert reader.docFreq(term) == st.docFreq;
@@ -539,6 +532,15 @@ public abstract class MultiTermQuery ext
       if (size != other.size) return false;
       return true;
     }
+    
+    private static final Comparator<ScoreTerm> scoreTermSortByTermComp = 
+      new Comparator<ScoreTerm>() {
+        public int compare(ScoreTerm st1, ScoreTerm st2) {
+          assert st1.termComp == st2.termComp :
+            "term comparator should not change between segments";
+          return st1.termComp.compare(st1.bytes, st2.bytes);
+        }
+      };
   
     static final class ScoreTerm implements Comparable<ScoreTerm> {
       public final Comparator<BytesRef> termComp;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PhraseQuery.java?rev=1027998&r1=1027997&r2=1027998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PhraseQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PhraseQuery.java Wed Oct 27 15:16:56 2010
@@ -20,13 +20,13 @@ package org.apache.lucene.search;
 import java.io.IOException;
 import java.util.Set;
 import java.util.ArrayList;
-import java.util.Arrays;
 
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.Explanation.IDFExplanation;
 import org.apache.lucene.util.ToStringUtils;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 
 /** A Query that matches documents containing a particular sequence of terms.
@@ -201,7 +201,7 @@ public class PhraseQuery extends Query {
 
       // sort by increasing docFreq order
       if (slop == 0) {
-        Arrays.sort(postingsFreqs);
+        ArrayUtil.quickSort(postingsFreqs);
       }
 
       if (slop == 0) {				  // optimize exact case

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/QueryTermVector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/QueryTermVector.java?rev=1027998&r1=1027997&r2=1027998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/QueryTermVector.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/QueryTermVector.java Wed Oct 27 15:16:56 2010
@@ -31,6 +31,7 @@ import org.apache.lucene.analysis.TokenS
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
 import org.apache.lucene.index.TermFreqVector;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 
 /**
@@ -81,7 +82,7 @@ public class QueryTermVector implements 
   
   private void processTerms(BytesRef[] queryTerms) {
     if (queryTerms != null) {
-      Arrays.sort(queryTerms);
+      ArrayUtil.quickSort(queryTerms);
       Map<BytesRef,Integer> tmpSet = new HashMap<BytesRef,Integer>(queryTerms.length);
       //filter out duplicates
       List<BytesRef> tmpList = new ArrayList<BytesRef>(queryTerms.length);

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java?rev=1027998&r1=1027997&r2=1027998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java Wed Oct 27 15:16:56 2010
@@ -18,10 +18,10 @@ package org.apache.lucene.search.spans;
  */
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.util.ArrayUtil;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Comparator;
 import java.util.HashSet;
 import java.util.LinkedList;
@@ -190,7 +190,7 @@ public class NearSpansOrdered extends Sp
 
   /** Advance the subSpans to the same document */
   private boolean toSameDoc() throws IOException {
-    Arrays.sort(subSpansByDoc, spanDocComparator);
+    ArrayUtil.quickSort(subSpansByDoc, spanDocComparator);
     int firstIndex = 0;
     int maxDoc = subSpansByDoc[subSpansByDoc.length - 1].doc();
     while (subSpansByDoc[firstIndex].doc() != maxDoc) {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/ArrayUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/ArrayUtil.java?rev=1027998&r1=1027997&r2=1027998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/ArrayUtil.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/ArrayUtil.java Wed Oct 27 15:16:56 2010
@@ -18,6 +18,7 @@ package org.apache.lucene.util;
  */
 
 import java.util.Collection;
+import java.util.Comparator;
 
 /**
  * Methods for manipulating arrays.
@@ -467,4 +468,177 @@ public final class ArrayUtil {
 
     return result;
   }
-}
+  
+  /** SorterTemplate with custom {@link Comparator} */
+  private static <T> SorterTemplate getSorter(final T[] a, final Comparator<? super T> comp) {
+    return new SorterTemplate() {
+      @Override
+      protected void swap(int i, int j) {
+        final T o = a[i];
+        a[i] = a[j];
+        a[j] = o;
+      }
+      
+      @Override
+      protected int compare(int i, int j) {
+        return comp.compare(a[i], a[j]);
+      }
+
+      @Override
+      protected void setPivot(int i) {
+        pivot = a[i];
+      }
+  
+      @Override
+      protected int comparePivot(int j) {
+        return comp.compare(pivot, a[j]);
+      }
+      
+      private T pivot;
+    };
+  }
+  
+  /** Natural SorterTemplate */
+  private static <T extends Comparable<? super T>> SorterTemplate getSorter(final T[] a) {
+    return new SorterTemplate() {
+      @Override
+      protected void swap(int i, int j) {
+        final T o = a[i];
+        a[i] = a[j];
+        a[j] = o;
+      }
+      
+      @Override
+      protected int compare(int i, int j) {
+        return a[i].compareTo(a[j]);
+      }
+
+      @Override
+      protected void setPivot(int i) {
+        pivot = a[i];
+      }
+  
+      @Override
+      protected int comparePivot(int j) {
+        return pivot.compareTo(a[j]);
+      }
+      
+      private T pivot;
+    };
+  }
+
+  // quickSorts (endindex is exclusive!):
+  
+  /**
+   * Sorts the given array slice using the {@link Comparator}. This method uses the quick sort
+   * algorithm, but falls back to insertion sort for small arrays.
+   * @param fromIndex start index (inclusive)
+   * @param toIndex end index (exclusive)
+   */
+  public static <T> void quickSort(T[] a, int fromIndex, int toIndex, Comparator<? super T> comp) {
+    getSorter(a, comp).quickSort(fromIndex, toIndex-1);
+  }
+  
+  /**
+   * Sorts the given array using the {@link Comparator}. This method uses the quick sort
+   * algorithm, but falls back to insertion sort for small arrays.
+   */
+  public static <T> void quickSort(T[] a, Comparator<? super T> comp) {
+    quickSort(a, 0, a.length, comp);
+  }
+  
+  /**
+   * Sorts the given array slice in natural order. This method uses the quick sort
+   * algorithm, but falls back to insertion sort for small arrays.
+   * @param fromIndex start index (inclusive)
+   * @param toIndex end index (exclusive)
+   */
+  public static <T extends Comparable<? super T>> void quickSort(T[] a, int fromIndex, int toIndex) {
+    getSorter(a).quickSort(fromIndex, toIndex-1);
+  }
+  
+  /**
+   * Sorts the given array in natural order. This method uses the quick sort
+   * algorithm, but falls back to insertion sort for small arrays.
+   */
+  public static <T extends Comparable<? super T>> void quickSort(T[] a) {
+    quickSort(a, 0, a.length);
+  }
+
+  // mergeSorts:
+  
+  /**
+   * Sorts the given array slice using the {@link Comparator}. This method uses the merge sort
+   * algorithm, but falls back to insertion sort for small arrays.
+   * @param fromIndex start index (inclusive)
+   * @param toIndex end index (exclusive)
+   */
+  public static <T> void mergeSort(T[] a, int fromIndex, int toIndex, Comparator<? super T> comp) {
+    getSorter(a, comp).mergeSort(fromIndex, toIndex-1);
+  }
+  
+  /**
+   * Sorts the given array using the {@link Comparator}. This method uses the merge sort
+   * algorithm, but falls back to insertion sort for small arrays.
+   */
+  public static <T> void mergeSort(T[] a, Comparator<? super T> comp) {
+    mergeSort(a, 0, a.length, comp);
+  }
+  
+  /**
+   * Sorts the given array slice in natural order. This method uses the merge sort
+   * algorithm, but falls back to insertion sort for small arrays.
+   * @param fromIndex start index (inclusive)
+   * @param toIndex end index (exclusive)
+   */
+  public static <T extends Comparable<? super T>> void mergeSort(T[] a, int fromIndex, int toIndex) {
+    getSorter(a).mergeSort(fromIndex, toIndex-1);
+  }
+  
+  /**
+   * Sorts the given array in natural order. This method uses the merge sort
+   * algorithm, but falls back to insertion sort for small arrays.
+   */
+  public static <T extends Comparable<? super T>> void mergeSort(T[] a) {
+    mergeSort(a, 0, a.length);
+  }
+
+  // insertionSorts:
+  
+  /**
+   * Sorts the given array slice using the {@link Comparator}. This method uses the insertion sort
+   * algorithm. It is only recommened to use this algorithm for partially sorted small arrays!
+   * @param fromIndex start index (inclusive)
+   * @param toIndex end index (exclusive)
+   */
+  public static <T> void insertionSort(T[] a, int fromIndex, int toIndex, Comparator<? super T> comp) {
+    getSorter(a, comp).insertionSort(fromIndex, toIndex-1);
+  }
+  
+  /**
+   * Sorts the given array using the {@link Comparator}. This method uses the insertion sort
+   * algorithm. It is only recommened to use this algorithm for partially sorted small arrays!
+   */
+  public static <T> void insertionSort(T[] a, Comparator<? super T> comp) {
+    insertionSort(a, 0, a.length, comp);
+  }
+  
+  /**
+   * Sorts the given array slice in natural order. This method uses the insertion sort
+   * algorithm. It is only recommened to use this algorithm for partially sorted small arrays!
+   * @param fromIndex start index (inclusive)
+   * @param toIndex end index (exclusive)
+   */
+  public static <T extends Comparable<? super T>> void insertionSort(T[] a, int fromIndex, int toIndex) {
+    getSorter(a).insertionSort(fromIndex, toIndex-1);
+  }
+  
+  /**
+   * Sorts the given array in natural order. This method uses the insertion sort
+   * algorithm. It is only recommened to use this algorithm for partially sorted small arrays!
+   */
+  public static <T extends Comparable<? super T>> void insertionSort(T[] a) {
+    insertionSort(a, 0, a.length);
+  }
+
+}
\ No newline at end of file

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/BytesRefHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/BytesRefHash.java?rev=1027998&r1=1027997&r2=1027998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/BytesRefHash.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/BytesRefHash.java Wed Oct 27 15:16:56 2010
@@ -45,18 +45,23 @@ import org.apache.lucene.util.ByteBlockP
  */
 public final class BytesRefHash {
 
-  private final ByteBlockPool pool;
+  public static final int DEFAULT_CAPACITY = 16;
+
+  // the following fields are needed by comparator,
+  // so package private to prevent access$-methods:
+  final ByteBlockPool pool;
+  int[] bytesStart;
+
+  private final BytesRef scratch1 = new BytesRef();
   private int hashSize;
   private int hashHalfSize;
   private int hashMask;
   private int count;
   private int lastCount = -1;
   private int[] ords;
-  private int[] bytesStart;
-  public static final int DEFAULT_CAPACITY = 16;
   private final BytesStartArray bytesStartArray;
   private AtomicLong bytesUsed;
-  
+
   /**
    * Creates a new {@link BytesRefHash} with a {@link ByteBlockPool} using a
    * {@link DirectAllocator}.
@@ -152,84 +157,49 @@ public final class BytesRefHash {
    * @param comp
    *          the {@link Comparator} used for sorting
    */
-  public int[] sort(Comparator<BytesRef> comp) {
+  public int[] sort(final Comparator<BytesRef> comp) {
     final int[] compact = compact();
-    quickSort(comp, compact, 0, count - 1);
-    return compact;
-  }
-
-  private void quickSort(Comparator<BytesRef> comp, int[] entries, int lo,
-      int hi) {
-    if (lo >= hi)
-      return;
-    if (hi == 1 + lo) {
-      if (compare(comp, entries[lo], entries[hi]) > 0) {
-        final int tmp = entries[lo];
-        entries[lo] = entries[hi];
-        entries[hi] = tmp;
-      }
-      return;
-    }
-    final int mid = (lo + hi) >>> 1;
-    if (compare(comp, entries[lo], entries[mid]) > 0) {
-      int tmp = entries[lo];
-      entries[lo] = entries[mid];
-      entries[mid] = tmp;
-    }
-
-    if (compare(comp, entries[mid], entries[hi]) > 0) {
-      int tmp = entries[mid];
-      entries[mid] = entries[hi];
-      entries[hi] = tmp;
-
-      if (compare(comp, entries[lo], entries[mid]) > 0) {
-        int tmp2 = entries[lo];
-        entries[lo] = entries[mid];
-        entries[mid] = tmp2;
-      }
-    }
-    int left = lo + 1;
-    int right = hi - 1;
-
-    if (left >= right)
-      return;
-
-    final int partition = entries[mid];
-
-    for (;;) {
-      while (compare(comp, entries[right], partition) > 0)
-        --right;
-
-      while (left < right && compare(comp, entries[left], partition) <= 0)
-        ++left;
-
-      if (left < right) {
-        final int tmp = entries[left];
-        entries[left] = entries[right];
-        entries[right] = tmp;
-        --right;
-      } else {
-        break;
+    new SorterTemplate() {
+      @Override
+      protected void swap(int i, int j) {
+        final int o = compact[i];
+        compact[i] = compact[j];
+        compact[j] = o;
+      }
+      
+      @Override
+      protected int compare(int i, int j) {
+        final int ord1 = compact[i], ord2 = compact[j];
+        assert bytesStart.length > ord1 && bytesStart.length > ord2;
+        return comp.compare(pool.setBytesRef(scratch1, bytesStart[ord1]),
+          pool.setBytesRef(scratch2, bytesStart[ord2]));
+      }
+
+      @Override
+      protected void setPivot(int i) {
+        final int ord = compact[i];
+        assert bytesStart.length > ord;
+        pool.setBytesRef(pivot, bytesStart[ord]);
       }
-    }
-
-    quickSort(comp, entries, lo, left);
-    quickSort(comp, entries, left + 1, hi);
+  
+      @Override
+      protected int comparePivot(int j) {
+        final int ord = compact[j];
+        assert bytesStart.length > ord;
+        return comp.compare(pivot,
+          pool.setBytesRef(scratch2, bytesStart[ord]));
+      }
+      
+      private final BytesRef pivot = new BytesRef(),
+        scratch1 = new BytesRef(), scratch2 = new BytesRef();
+    }.quickSort(0, count - 1);
+    return compact;
   }
 
-  private final BytesRef scratch1 = new BytesRef();
-  private final BytesRef scratch2 = new BytesRef();
-
   private boolean equals(int ord, BytesRef b) {
     return pool.setBytesRef(scratch1, bytesStart[ord]).bytesEquals(b);
   }
 
-  private int compare(Comparator<BytesRef> comp, int ord1, int ord2) {
-    assert bytesStart.length > ord1 && bytesStart.length > ord2;
-    return comp.compare(pool.setBytesRef(scratch1, bytesStart[ord1]),
-        pool.setBytesRef(scratch2, bytesStart[ord2]));
-  }
-
   private boolean shrink(int targetSize) {
     // Cannot use ArrayUtil.shrink because we require power
     // of 2:

Added: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/CollectionUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/CollectionUtil.java?rev=1027998&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/CollectionUtil.java (added)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/CollectionUtil.java Wed Oct 27 15:16:56 2010
@@ -0,0 +1,160 @@
+package org.apache.lucene.util;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.Comparator;
+import java.util.Collections;
+import java.util.List;
+import java.util.RandomAccess;
+
+/**
+ * Methods for manipulating (sorting) collections.
+ * Sort methods work directly on the supplied lists and don't copy to/from arrays
+ * before/after. For medium size collections as used in the Lucene indexer that is
+ * much more efficient.
+ *
+ * @lucene.internal
+ */
+
+public final class CollectionUtil {
+
+  private CollectionUtil() {} // no instance
+  
+  /** SorterTemplate with custom {@link Comparator} */
+  private static <T> SorterTemplate getSorter(final List<T> list, final Comparator<? super T> comp) {
+    if (!(list instanceof RandomAccess))
+      throw new IllegalArgumentException("CollectionUtil can only sort random access lists in-place.");
+    return new SorterTemplate() {
+      @Override
+      protected void swap(int i, int j) {
+        Collections.swap(list, i, j);
+      }
+      
+      @Override
+      protected int compare(int i, int j) {
+        return comp.compare(list.get(i), list.get(j));
+      }
+
+      @Override
+      protected void setPivot(int i) {
+        pivot = list.get(i);
+      }
+  
+      @Override
+      protected int comparePivot(int j) {
+        return comp.compare(pivot, list.get(j));
+      }
+      
+      private T pivot;
+    };
+  }
+  
+  /** Natural SorterTemplate */
+  private static <T extends Comparable<? super T>> SorterTemplate getSorter(final List<T> list) {
+    if (!(list instanceof RandomAccess))
+      throw new IllegalArgumentException("CollectionUtil can only sort random access lists in-place.");
+    return new SorterTemplate() {
+      @Override
+      protected void swap(int i, int j) {
+        Collections.swap(list, i, j);
+      }
+      
+      @Override
+      protected int compare(int i, int j) {
+        return list.get(i).compareTo(list.get(j));
+      }
+
+      @Override
+      protected void setPivot(int i) {
+        pivot = list.get(i);
+      }
+  
+      @Override
+      protected int comparePivot(int j) {
+        return pivot.compareTo(list.get(j));
+      }
+      
+      private T pivot;
+    };
+  }
+
+  /**
+   * Sorts the given random access {@link List} using the {@link Comparator}.
+   * The list must implement {@link RandomAccess}. This method uses the quick sort
+   * algorithm, but falls back to insertion sort for small lists.
+   * @throws IllegalArgumentException if list is e.g. a linked list without random access.
+   */
+  public static <T> void quickSort(List<T> list, Comparator<? super T> comp) {
+    getSorter(list, comp).quickSort(0, list.size()-1);
+  }
+  
+  /**
+   * Sorts the given random access {@link List} in natural order.
+   * The list must implement {@link RandomAccess}. This method uses the quick sort
+   * algorithm, but falls back to insertion sort for small lists.
+   * @throws IllegalArgumentException if list is e.g. a linked list without random access.
+   */
+  public static <T extends Comparable<? super T>> void quickSort(List<T> list) {
+    getSorter(list).quickSort(0, list.size()-1);
+  }
+
+  // mergeSorts:
+  
+  /**
+   * Sorts the given random access {@link List} using the {@link Comparator}.
+   * The list must implement {@link RandomAccess}. This method uses the merge sort
+   * algorithm, but falls back to insertion sort for small lists.
+   * @throws IllegalArgumentException if list is e.g. a linked list without random access.
+   */
+  public static <T> void mergeSort(List<T> list, Comparator<? super T> comp) {
+    getSorter(list, comp).mergeSort(0, list.size()-1);
+  }
+  
+  /**
+   * Sorts the given random access {@link List} in natural order.
+   * The list must implement {@link RandomAccess}. This method uses the merge sort
+   * algorithm, but falls back to insertion sort for small lists.
+   * @throws IllegalArgumentException if list is e.g. a linked list without random access.
+   */
+  public static <T extends Comparable<? super T>> void mergeSort(List<T> list) {
+    getSorter(list).mergeSort(0, list.size()-1);
+  }
+
+  // insertionSorts:
+  
+  /**
+   * Sorts the given random access {@link List} using the {@link Comparator}.
+   * The list must implement {@link RandomAccess}. This method uses the insertion sort
+   * algorithm. It is only recommened to use this algorithm for partially sorted small lists!
+   * @throws IllegalArgumentException if list is e.g. a linked list without random access.
+   */
+  public static <T> void insertionSort(List<T> list, Comparator<? super T> comp) {
+    getSorter(list, comp).insertionSort(0, list.size()-1);
+  }
+  
+  /**
+   * Sorts the given random access {@link List} in natural order.
+   * The list must implement {@link RandomAccess}. This method uses the insertion sort
+   * algorithm. It is only recommened to use this algorithm for partially sorted small lists!
+   * @throws IllegalArgumentException if list is e.g. a linked list without random access.
+   */
+  public static <T extends Comparable<? super T>> void insertionSort(List<T> list) {
+    getSorter(list).insertionSort(0, list.size()-1);
+  }
+  
+}
\ No newline at end of file

Propchange: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/CollectionUtil.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/CollectionUtil.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/SorterTemplate.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/SorterTemplate.java?rev=1027998&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/SorterTemplate.java (added)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/SorterTemplate.java Wed Oct 27 15:16:56 2010
@@ -0,0 +1,199 @@
+package org.apache.lucene.util;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * This class was inspired by CGLIB, but provides a better
+ * QuickSort algorithm without additional InsertionSort
+ * at the end.
+ * To use, subclass and override the four abstract methods
+ * which compare and modify your data.
+ * Allows custom swap so that two arrays can be sorted
+ * at the same time.
+ * @lucene.internal
+ */
+public abstract class SorterTemplate {
+
+  private static final int MERGESORT_THRESHOLD = 12;
+  private static final int QUICKSORT_THRESHOLD = 7;
+
+  /** Implement this method, that swaps slots {@code i} and {@code j} in your data */
+  protected abstract void swap(int i, int j);
+  
+  /** Compares slots {@code i} and {@code j} of you data.
+   * Should be implemented like <code><em>valueOf(j)</em>.compareTo(<em>valueOf(i)</em>)</code> */
+  protected abstract int compare(int i, int j);
+
+  /** Implement this method, that stores the value of slot {@code i} as pivot value */
+  protected abstract void setPivot(int i);
+  
+  /** Implements the compare function for the previously stored pivot value.
+   * Should be implemented like <code>pivot.compareTo(<em>valueOf(i)</em>)</code> */
+  protected abstract int comparePivot(int j);
+  
+  /** Sorts via stable in-place InsertionSort algorithm
+   *(ideal for small collections which are mostly presorted). */
+  public final void insertionSort(int lo, int hi) {
+    for (int i = lo + 1 ; i <= hi; i++) {
+      for (int j = i; j > lo; j--) {
+        if (compare(j - 1, j) > 0) {
+          swap(j - 1, j);
+        } else {
+          break;
+        }
+      }
+    }
+  }
+
+  /** Sorts via in-place, but unstable, QuickSort algorithm.
+   * For small collections falls back to {@link #insertionSort(int,int)}. */
+  public final void quickSort(int lo, int hi) {
+    final int diff = hi - lo;
+    if (diff <= QUICKSORT_THRESHOLD) {
+      insertionSort(lo, hi);
+      return;
+    }
+    
+    final int mid = lo + (diff >>> 1);
+    
+    if (compare(lo, mid) > 0) {
+      swap(lo, mid);
+    }
+
+    if (compare(mid, hi) > 0) {
+      swap(mid, hi);
+      if (compare(lo, mid) > 0) {
+        swap(lo, mid);
+      }
+    }
+    
+    int left = lo + 1;
+    int right = hi - 1;
+
+    setPivot(mid);
+    for (;;) {
+      while (comparePivot(right) < 0)
+        --right;
+
+      while (left < right && comparePivot(left) >= 0)
+        ++left;
+
+      if (left < right) {
+        swap(left, right);
+        --right;
+      } else {
+        break;
+      }
+    }
+
+    quickSort(lo, left);
+    quickSort(left + 1, hi);
+  }
+  
+  /** Sorts via stable in-place MergeSort algorithm
+   * For small collections falls back to {@link #insertionSort(int,int)}. */
+  public final void mergeSort(int lo, int hi) {
+    final int diff = hi - lo;
+    if (diff <= MERGESORT_THRESHOLD) {
+      insertionSort(lo, hi);
+      return;
+    }
+    
+    final int mid = lo + (diff >>> 1);
+    
+    mergeSort(lo, mid);
+    mergeSort(mid, hi);
+    merge(lo, mid, hi, mid - lo, hi - mid);
+  }
+
+  private void merge(int lo, int pivot, int hi, int len1, int len2) {
+    if (len1 == 0 || len2 == 0) {
+      return;
+    }
+    if (len1 + len2 == 2) {
+      if (compare(pivot, lo) < 0) {
+          swap(pivot, lo);
+      }
+      return;
+    }
+    int first_cut, second_cut;
+    int len11, len22;
+    if (len1 > len2) {
+      len11 = len1 >>> 1;
+      first_cut = lo + len11;
+      second_cut = lower(pivot, hi, first_cut);
+      len22 = second_cut - pivot;
+    } else {
+      len22 = len2 >>> 1;
+      second_cut = pivot + len22;
+      first_cut = upper(lo, pivot, second_cut);
+      len11 = first_cut - lo;
+    }
+    rotate(first_cut, pivot, second_cut);
+    final int new_mid = first_cut + len22;
+    merge(lo, first_cut, new_mid, len11, len22);
+    merge(new_mid, second_cut, hi, len1 - len11, len2 - len22);
+  }
+
+  private void rotate(int lo, int mid, int hi) {
+    int lot = lo;
+    int hit = mid - 1;
+    while (lot < hit) {
+      swap(lot++, hit--);
+    }
+    lot = mid; hit = hi - 1;
+    while (lot < hit) {
+      swap(lot++, hit--);
+    }
+    lot = lo; hit = hi - 1;
+    while (lot < hit) {
+      swap(lot++, hit--);
+    }
+  }
+
+  private int lower(int lo, int hi, int val) {
+    int len = hi - lo;
+    while (len > 0) {
+      final int half = len >>> 1,
+        mid = lo + half;
+      if (compare(mid, val) < 0) {
+        lo = mid + 1;
+        len = len - half -1;
+      } else {
+        len = half;
+      }
+    }
+    return lo;
+  }
+
+  private int upper(int lo, int hi, int val) {
+    int len = hi - lo;
+    while (len > 0) {
+      final int half = len >>> 1,
+        mid = lo + half;
+      if (compare(val, mid) < 0) {
+        len = half;
+      } else {
+        lo = mid + 1;
+        len = len - half -1;
+      }
+    }
+    return lo;
+  }
+
+}

Propchange: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/SorterTemplate.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/SorterTemplate.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/BasicOperations.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/BasicOperations.java?rev=1027998&r1=1027997&r2=1027998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/BasicOperations.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/BasicOperations.java Wed Oct 27 15:16:56 2010
@@ -34,7 +34,6 @@ import org.apache.lucene.util.RamUsageEs
 
 import java.util.ArrayList;
 import java.util.BitSet;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -555,9 +554,8 @@ final public class BasicOperations {
     }
 
     public void sort() {
-      if (count > 1) {
-        Arrays.sort(points, 0, count);
-      }
+      // mergesort seems to perform better on already sorted arrays:
+      if (count > 1) ArrayUtil.mergeSort(points, 0, count);
     }
 
     public void add(Transition t) {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/State.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/State.java?rev=1027998&r1=1027997&r2=1027998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/State.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/State.java Wed Oct 27 15:16:56 2010
@@ -32,7 +32,6 @@ import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.RamUsageEstimator;
 
 import java.io.Serializable;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.Iterator;
@@ -232,8 +231,8 @@ public class State implements Serializab
   
   /** Sorts transitions array in-place. */
   public void sortTransitions(Comparator<Transition> comparator) {
-    if (numTransitions > 1)
-      Arrays.sort(transitionsArray, 0, numTransitions, comparator);
+    // mergesort seems to perform better on already sorted arrays:
+    if (numTransitions > 1) ArrayUtil.mergeSort(transitionsArray, 0, numTransitions, comparator);
   }
   
   /**

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestArrayUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestArrayUtil.java?rev=1027998&r1=1027997&r2=1027998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestArrayUtil.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestArrayUtil.java Wed Oct 27 15:16:56 2010
@@ -17,6 +17,9 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
+import java.util.Arrays;
+import java.util.Collections;
+
 public class TestArrayUtil extends LuceneTestCase {
 
   // Ensure ArrayUtil.getNextSize gives linear amortized cost of realloc/copy
@@ -113,4 +116,127 @@ public class TestArrayUtil extends Lucen
     assertFalse(left + " does not equal: " + right, ArrayUtil.equals(leftChars, 25, rightChars, 0, left.length()));
     assertFalse(left + " does not equal: " + right, ArrayUtil.equals(leftChars, 12, rightChars, 0, left.length()));
   }
+  
+  private Integer[] createRandomArray(int maxSize) {
+    final Integer[] a = new Integer[random.nextInt(maxSize) + 1];
+    for (int i = 0; i < a.length; i++) {
+      a[i] = Integer.valueOf(random.nextInt(a.length));
+    }
+    return a;
+  }
+  
+  public void testQuickSort() {
+    for (int i = 0, c = 500 * RANDOM_MULTIPLIER; i < c; i++) {
+      Integer[] a1 = createRandomArray(1000), a2 = a1.clone();
+      ArrayUtil.quickSort(a1);
+      Arrays.sort(a2);
+      assertArrayEquals(a2, a1);
+      
+      a1 = createRandomArray(1000);
+      a2 = a1.clone();
+      ArrayUtil.quickSort(a1, Collections.reverseOrder());
+      Arrays.sort(a2, Collections.reverseOrder());
+      assertArrayEquals(a2, a1);
+      // reverse back, so we can test that completely backwards sorted array (worst case) is working:
+      ArrayUtil.quickSort(a1);
+      Arrays.sort(a2);
+      assertArrayEquals(a2, a1);
+    }
+  }
+  
+  public void testMergeSort() {
+    for (int i = 0, c = 500 * RANDOM_MULTIPLIER; i < c; i++) {
+      Integer[] a1 = createRandomArray(1000), a2 = a1.clone();
+      ArrayUtil.mergeSort(a1);
+      Arrays.sort(a2);
+      assertArrayEquals(a2, a1);
+      
+      a1 = createRandomArray(1000);
+      a2 = a1.clone();
+      ArrayUtil.mergeSort(a1, Collections.reverseOrder());
+      Arrays.sort(a2, Collections.reverseOrder());
+      assertArrayEquals(a2, a1);
+      // reverse back, so we can test that completely backwards sorted array (worst case) is working:
+      ArrayUtil.mergeSort(a1);
+      Arrays.sort(a2);
+      assertArrayEquals(a2, a1);
+    }
+  }
+  
+  public void testInsertionSort() {
+    for (int i = 0, c = 500 * RANDOM_MULTIPLIER; i < c; i++) {
+      Integer[] a1 = createRandomArray(30), a2 = a1.clone();
+      ArrayUtil.insertionSort(a1);
+      Arrays.sort(a2);
+      assertArrayEquals(a2, a1);
+      
+      a1 = createRandomArray(30);
+      a2 = a1.clone();
+      ArrayUtil.insertionSort(a1, Collections.reverseOrder());
+      Arrays.sort(a2, Collections.reverseOrder());
+      assertArrayEquals(a2, a1);
+      // reverse back, so we can test that completely backwards sorted array (worst case) is working:
+      ArrayUtil.insertionSort(a1);
+      Arrays.sort(a2);
+      assertArrayEquals(a2, a1);
+    }
+  }
+  
+  static class Item implements Comparable<Item> {
+    final int val, order;
+    
+    Item(int val, int order) {
+      this.val = val;
+      this.order = order;
+    }
+    
+    public int compareTo(Item other) {
+      return this.order - other.order;
+    }
+    
+    @Override
+    public String toString() {
+      return Integer.toString(val);
+    }
+  }
+  
+  public void testMergeSortStability() {
+    Item[] items = new Item[100];
+    for (int i = 0; i < items.length; i++) {
+      // half of the items have value but same order. The value of this items is sorted,
+      // so they should always be in order after sorting.
+      // The other half has defined order, but no (-1) value (they should appear after
+      // all above, when sorted).
+      final boolean equal = random.nextBoolean();
+      items[i] = new Item(equal ? (i+1) : -1, equal ? 0 : (random.nextInt(1000)+1));
+    }
+    
+    if (VERBOSE) System.out.println("Before: " + Arrays.toString(items));
+    // if you replace this with ArrayUtil.quickSort(), test should fail:
+    ArrayUtil.mergeSort(items);
+    if (VERBOSE) System.out.println("Sorted: " + Arrays.toString(items));
+    
+    Item last = items[0];
+    for (int i = 1; i < items.length; i++) {
+      final Item act = items[i];
+      if (act.order == 0) {
+        // order of "equal" items should be not mixed up
+        assertTrue(act.val > last.val);
+      }
+      assertTrue(act.order >= last.order);
+      last = act;
+    }
+  }
+  
+  // should produce no exceptions
+  public void testEmptyArraySort() {
+    Integer[] a = new Integer[0];
+    ArrayUtil.quickSort(a);
+    ArrayUtil.mergeSort(a);
+    ArrayUtil.insertionSort(a);
+    ArrayUtil.quickSort(a, Collections.reverseOrder());
+    ArrayUtil.mergeSort(a, Collections.reverseOrder());
+    ArrayUtil.insertionSort(a, Collections.reverseOrder());
+  }
+  
 }

Added: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestCollectionUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestCollectionUtil.java?rev=1027998&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestCollectionUtil.java (added)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestCollectionUtil.java Wed Oct 27 15:16:56 2010
@@ -0,0 +1,103 @@
+package org.apache.lucene.util;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class TestCollectionUtil extends LuceneTestCase {
+
+  private List<Integer> createRandomList(int maxSize) {
+    final Integer[] a = new Integer[random.nextInt(maxSize) + 1];
+    for (int i = 0; i < a.length; i++) {
+      a[i] = Integer.valueOf(random.nextInt(a.length));
+    }
+    return Arrays.asList(a);
+  }
+  
+  public void testQuickSort() {
+    for (int i = 0, c = 500 * RANDOM_MULTIPLIER; i < c; i++) {
+      List<Integer> list1 = createRandomList(1000), list2 = new ArrayList<Integer>(list1);
+      CollectionUtil.quickSort(list1);
+      Collections.sort(list2);
+      assertEquals(list2, list1);
+      
+      list1 = createRandomList(1000);
+      list2 = new ArrayList<Integer>(list1);
+      CollectionUtil.quickSort(list1, Collections.reverseOrder());
+      Collections.sort(list2, Collections.reverseOrder());
+      assertEquals(list2, list1);
+      // reverse back, so we can test that completely backwards sorted array (worst case) is working:
+      CollectionUtil.quickSort(list1);
+      Collections.sort(list2);
+      assertEquals(list2, list1);
+    }
+  }
+  
+  public void testMergeSort() {
+    for (int i = 0, c = 500 * RANDOM_MULTIPLIER; i < c; i++) {
+      List<Integer> list1 = createRandomList(1000), list2 = new ArrayList<Integer>(list1);
+      CollectionUtil.mergeSort(list1);
+      Collections.sort(list2);
+      assertEquals(list2, list1);
+      
+      list1 = createRandomList(1000);
+      list2 = new ArrayList<Integer>(list1);
+      CollectionUtil.mergeSort(list1, Collections.reverseOrder());
+      Collections.sort(list2, Collections.reverseOrder());
+      assertEquals(list2, list1);
+      // reverse back, so we can test that completely backwards sorted array (worst case) is working:
+      CollectionUtil.mergeSort(list1);
+      Collections.sort(list2);
+      assertEquals(list2, list1);
+    }
+  }
+  
+  public void testInsertionSort() {
+    for (int i = 0, c = 500 * RANDOM_MULTIPLIER; i < c; i++) {
+      List<Integer> list1 = createRandomList(30), list2 = new ArrayList<Integer>(list1);
+      CollectionUtil.insertionSort(list1);
+      Collections.sort(list2);
+      assertEquals(list2, list1);
+      
+      list1 = createRandomList(30);
+      list2 = new ArrayList<Integer>(list1);
+      CollectionUtil.insertionSort(list1, Collections.reverseOrder());
+      Collections.sort(list2, Collections.reverseOrder());
+      assertEquals(list2, list1);
+      // reverse back, so we can test that completely backwards sorted array (worst case) is working:
+      CollectionUtil.insertionSort(list1);
+      Collections.sort(list2);
+      assertEquals(list2, list1);
+    }
+  }
+  
+  // should produce no exceptions
+  public void testEmptyArraySort() {
+    List<Integer> list = Collections.emptyList();
+    CollectionUtil.quickSort(list);
+    CollectionUtil.mergeSort(list);
+    CollectionUtil.insertionSort(list);
+    CollectionUtil.quickSort(list, Collections.reverseOrder());
+    CollectionUtil.mergeSort(list, Collections.reverseOrder());
+    CollectionUtil.insertionSort(list, Collections.reverseOrder());
+  }
+  
+}

Propchange: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestCollectionUtil.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestCollectionUtil.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL