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

svn commit: r1478785 [1/2] - in /lucene/dev/trunk: lucene/ lucene/core/src/java/org/apache/lucene/index/ lucene/core/src/java/org/apache/lucene/search/ lucene/core/src/java/org/apache/lucene/search/spans/ lucene/core/src/java/org/apache/lucene/util/ lu...

Author: jpountz
Date: Fri May  3 13:37:45 2013
New Revision: 1478785

URL: http://svn.apache.org/r1478785
Log:
LUCENE-4946: Refactor SorterTemplate (now Sorter).

Added:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/ArrayInPlaceMergeSorter.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/ArrayIntroSorter.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/ArrayTimSorter.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/InPlaceMergeSorter.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/IntroSorter.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/Sorter.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/TimSorter.java   (with props)
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/BaseSortTestCase.java   (with props)
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestInPlaceMergeSorter.java   (with props)
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestIntroSorter.java   (with props)
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestTimSorter.java   (with props)
Removed:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/SorterTemplate.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestSorterTemplate.java
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/NOTICE.txt
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopTermsRewrite.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/ArrayUtil.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/CollectionUtil.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/BasicOperations.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/State.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestArrayUtil.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestCollectionUtil.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestDrillSideways.java
    lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TokenSources.java
    lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TokenStreamFromTermPositionVector.java
    lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/Passage.java
    lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java
    lucene/dev/trunk/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
    lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/index/CompoundFileExtractor.java
    lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/index/sorter/Sorter.java
    lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingAtomicReader.java
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/BytesRefArray.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1478785&r1=1478784&r2=1478785&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Fri May  3 13:37:45 2013
@@ -55,6 +55,10 @@ Changes in backwards compatibility polic
 * LUCENE-4973: SnapshotDeletionPolicy no longer requires a unique
   String id (Mike McCandless, Shai Erera)
 
+* LUCENE-4946: The internal sorting API (SorterTemplate, now Sorter) has been
+  completely refactored to allow for a better implementation of TimSort.
+  (Adrien Grand, Uwe Schindler, Dawid Weiss)
+
 Bug Fixes
 
 * LUCENE-4935: CustomScoreQuery wrongly applied its query boost twice 

Modified: lucene/dev/trunk/lucene/NOTICE.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/NOTICE.txt?rev=1478785&r1=1478784&r2=1478785&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/NOTICE.txt (original)
+++ lucene/dev/trunk/lucene/NOTICE.txt Fri May  3 13:37:45 2013
@@ -27,11 +27,6 @@ Jean-Philippe Barrette-LaPierre. This li
 see http://sites.google.com/site/rrettesite/moman and 
 http://bitbucket.org/jpbarrette/moman/overview/
 
-The class org.apache.lucene.util.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.
-
 The class org.apache.lucene.util.WeakIdentityMap was derived from
 the Apache CXF project and is Apache License 2.0.
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java?rev=1478785&r1=1478784&r2=1478785&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java Fri May  3 13:37:45 2013
@@ -184,7 +184,7 @@ public class ConcurrentMergeScheduler ex
     }
 
     // Sort the merge threads in descending order.
-    CollectionUtil.mergeSort(activeMerges, compareByMergeDocCount);
+    CollectionUtil.timSort(activeMerges, compareByMergeDocCount);
     
     int pri = mergeThreadPriority;
     final int activeMergeCount = activeMerges.size();

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

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java?rev=1478785&r1=1478784&r2=1478785&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java Fri May  3 13:37:45 2013
@@ -54,7 +54,7 @@ final class FreqProxTermsWriter extends 
     final int numAllFields = allFields.size();
 
     // Sort by field name
-    CollectionUtil.quickSort(allFields);
+    CollectionUtil.introSort(allFields);
 
     final FieldsConsumer consumer = state.segmentInfo.getCodec().postingsFormat().fieldsConsumer(state);
 

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

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=1478785&r1=1478784&r2=1478785&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java Fri May  3 13:37:45 2013
@@ -232,7 +232,7 @@ final class IndexFileDeleter implements 
     }
 
     // We keep commits list in sorted order (oldest to newest):
-    CollectionUtil.mergeSort(commits);
+    CollectionUtil.timSort(commits);
 
     // Now delete anything with ref count at 0.  These are
     // presumably abandoned files eg due to crash of

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java?rev=1478785&r1=1478784&r2=1478785&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java Fri May  3 13:37:45 2013
@@ -44,7 +44,7 @@ class ConjunctionScorer extends Scorer {
     }
     // Sort the array the first time to allow the least frequent DocsEnum to
     // lead the matching.
-    ArrayUtil.mergeSort(docsAndFreqs, new Comparator<DocsAndFreqs>() {
+    ArrayUtil.timSort(docsAndFreqs, new Comparator<DocsAndFreqs>() {
       @Override
       public int compare(DocsAndFreqs o1, DocsAndFreqs o2) {
         return Long.compare(o1.cost, o2.cost);

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java?rev=1478785&r1=1478784&r2=1478785&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java Fri May  3 13:37:45 2013
@@ -87,7 +87,7 @@ class MinShouldMatchSumScorer extends Sc
     this.sortedSubScorers = subScorers.toArray(new Scorer[this.numScorers]);
     // sorting by decreasing subscorer cost should be inversely correlated with
     // next docid (assuming costs are due to generating many postings)
-    ArrayUtil.mergeSort(sortedSubScorers, new Comparator<Scorer>() {
+    ArrayUtil.timSort(sortedSubScorers, new Comparator<Scorer>() {
       @Override
       public int compare(Scorer o1, Scorer o2) {
         return Long.signum(o2.cost() - o1.cost());

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java?rev=1478785&r1=1478784&r2=1478785&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java Fri May  3 13:37:45 2013
@@ -241,7 +241,7 @@ public class MultiPhraseQuery extends Qu
 
       // sort by increasing docFreq order
       if (slop == 0) {
-        ArrayUtil.mergeSort(postingsFreqs);
+        ArrayUtil.timSort(postingsFreqs);
       }
 
       if (slop == 0) {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java?rev=1478785&r1=1478784&r2=1478785&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java Fri May  3 13:37:45 2013
@@ -278,7 +278,7 @@ public class PhraseQuery extends Query {
 
       // sort by increasing docFreq order
       if (slop == 0) {
-        ArrayUtil.mergeSort(postingsFreqs);
+        ArrayUtil.timSort(postingsFreqs);
       }
 
       if (slop == 0) {  // optimize exact case

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopTermsRewrite.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopTermsRewrite.java?rev=1478785&r1=1478784&r2=1478785&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopTermsRewrite.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopTermsRewrite.java Fri May  3 13:37:45 2013
@@ -156,7 +156,7 @@ public abstract class TopTermsRewrite<Q 
     
     final Q q = getTopLevelQuery();
     final ScoreTerm[] scoreTerms = stQueue.toArray(new ScoreTerm[stQueue.size()]);
-    ArrayUtil.mergeSort(scoreTerms, scoreTermSortByTermComp);
+    ArrayUtil.timSort(scoreTerms, scoreTermSortByTermComp);
     
     for (final ScoreTerm st : scoreTerms) {
       final Term term = new Term(query.field, st.bytes);

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java?rev=1478785&r1=1478784&r2=1478785&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java Fri May  3 13:37:45 2013
@@ -204,7 +204,7 @@ public class NearSpansOrdered extends Sp
 
   /** Advance the subSpans to the same document */
   private boolean toSameDoc() throws IOException {
-    ArrayUtil.mergeSort(subSpansByDoc, spanDocComparator);
+    ArrayUtil.timSort(subSpansByDoc, spanDocComparator);
     int firstIndex = 0;
     int maxDoc = subSpansByDoc[subSpansByDoc.length - 1].doc();
     while (subSpansByDoc[firstIndex].doc() != maxDoc) {

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/ArrayInPlaceMergeSorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/ArrayInPlaceMergeSorter.java?rev=1478785&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/ArrayInPlaceMergeSorter.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/ArrayInPlaceMergeSorter.java Fri May  3 13:37:45 2013
@@ -0,0 +1,47 @@
+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;
+
+/**
+ * An {@link InPlaceMergeSorter} for object arrays.
+ * @lucene.internal
+ */
+final class ArrayInPlaceMergeSorter<T> extends InPlaceMergeSorter {
+
+  private final T[] arr;
+  private final Comparator<? super T> comparator;
+
+  /** Create a new {@link ArrayInPlaceMergeSorter}. */
+  public ArrayInPlaceMergeSorter(T[] arr, Comparator<? super T> comparator) {
+    this.arr = arr;
+    this.comparator = comparator;
+  }
+
+  @Override
+  protected int compare(int i, int j) {
+    return comparator.compare(arr[i], arr[j]);
+  }
+
+  @Override
+  protected void swap(int i, int j) {
+    ArrayUtil.swap(arr, i, j);
+  }
+
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/ArrayIntroSorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/ArrayIntroSorter.java?rev=1478785&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/ArrayIntroSorter.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/ArrayIntroSorter.java Fri May  3 13:37:45 2013
@@ -0,0 +1,59 @@
+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;
+
+/**
+ * An {@link IntroSorter} for object arrays.
+ * @lucene.internal
+ */
+final class ArrayIntroSorter<T> extends IntroSorter {
+
+  private final T[] arr;
+  private final Comparator<? super T> comparator;
+  private T pivot;
+
+  /** Create a new {@link ArrayInPlaceMergeSorter}. */
+  public ArrayIntroSorter(T[] arr, Comparator<? super T> comparator) {
+    this.arr = arr;
+    this.comparator = comparator;
+    pivot = null;
+  }
+
+  @Override
+  protected int compare(int i, int j) {
+    return comparator.compare(arr[i], arr[j]);
+  }
+
+  @Override
+  protected void swap(int i, int j) {
+    ArrayUtil.swap(arr, i, j);
+  }
+
+  @Override
+  protected void setPivot(int i) {
+    pivot = arr[i];
+  }
+
+  @Override
+  protected int comparePivot(int i) {
+    return comparator.compare(pivot, arr[i]);
+  }
+
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/ArrayTimSorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/ArrayTimSorter.java?rev=1478785&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/ArrayTimSorter.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/ArrayTimSorter.java Fri May  3 13:37:45 2013
@@ -0,0 +1,76 @@
+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;
+
+/**
+ * A {@link TimSorter} for object arrays.
+ * @lucene.internal
+ */
+final class ArrayTimSorter<T> extends TimSorter {
+
+  private final Comparator<? super T> comparator;
+  private final T[] arr;
+  private final T[] tmp;
+
+  /** Create a new {@link ArrayTimSorter}. */
+  public ArrayTimSorter(T[] arr, Comparator<? super T> comparator, int maxTempSlots) {
+    super(maxTempSlots);
+    this.arr = arr;
+    this.comparator = comparator;
+    if (maxTempSlots > 0) {
+      @SuppressWarnings("unchecked")
+      final T[] tmp = (T[]) new Object[maxTempSlots];
+      this.tmp = tmp;
+    } else {
+      this.tmp = null;
+    }
+  }
+
+  @Override
+  protected int compare(int i, int j) {
+    return comparator.compare(arr[i], arr[j]);
+  }
+
+  @Override
+  protected void swap(int i, int j) {
+    ArrayUtil.swap(arr, i, j);
+  }
+
+  @Override
+  protected void copy(int src, int dest) {
+    arr[dest] = arr[src];
+  }
+
+  @Override
+  protected void save(int start, int len) {
+    System.arraycopy(arr, start, tmp, 0, len);
+  }
+
+  @Override
+  protected void restore(int src, int dest) {
+    arr[dest] = tmp[src];
+  }
+
+  @Override
+  protected int compareSaved(int i, int j) {
+    return comparator.compare(tmp[i], arr[j]);
+  }
+
+}

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/ArrayUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/ArrayUtil.java?rev=1478785&r1=1478784&r2=1478785&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/ArrayUtil.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/ArrayUtil.java Fri May  3 13:37:45 2013
@@ -28,11 +28,6 @@ import java.util.Comparator;
 
 public final class ArrayUtil {
 
-  // affordable memory overhead to merge sorted arrays
-  static final float MERGE_OVERHEAD_RATIO = 0.01f;
-  // arrays below this size will always be sorted in-place
-  static final int MERGE_EXTRA_MEMORY_THRESHOLD = (int) (15 / MERGE_OVERHEAD_RATIO);
-
   private ArrayUtil() {} // no instance
 
   /*
@@ -610,237 +605,85 @@ public final class ArrayUtil {
     return result;
   }
 
-  private static abstract class ArraySorterTemplate<T> extends SorterTemplate {
-
-    protected final T[] a;
-
-    ArraySorterTemplate(T[] a) {
-      this.a = a;
-    }
-
-    protected abstract int compare(T a, T b);
-
-    @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 compare(a[i], a[j]);
-    }
-
+  private static class NaturalComparator<T extends Comparable<? super T>> implements Comparator<T> {
+    NaturalComparator() {}
     @Override
-    protected void setPivot(int i) {
-      pivot = a[i];
-    }
-
-    @Override
-    protected int comparePivot(int j) {
-      return compare(pivot, a[j]);
-    }
-
-    private T pivot;
-
-  }
-
-  // a template for merge-based sorts which uses extra memory to speed up merging
-  private static abstract class ArrayMergeSorterTemplate<T> extends ArraySorterTemplate<T> {
-
-    private final int threshold; // maximum length of a merge that can be made using extra memory
-    private final T[] tmp;
-
-    ArrayMergeSorterTemplate(T[] a, float overheadRatio) {
-      super(a);
-      this.threshold = (int) (a.length * overheadRatio);
-      @SuppressWarnings("unchecked")
-      final T[] tmpBuf = (T[]) new Object[threshold];
-      this.tmp = tmpBuf;
-    }
-
-    private void mergeWithExtraMemory(int lo, int pivot, int hi, int len1, int len2) {
-      System.arraycopy(a, lo, tmp, 0, len1);
-      int i = 0, j = pivot, dest = lo;
-      while (i < len1 && j < hi) {
-        if (compare(tmp[i], a[j]) <= 0) {
-          a[dest++] = tmp[i++];
-        } else {
-          a[dest++] = a[j++];
-        }
-      }
-      while (i < len1) {
-        a[dest++] = tmp[i++];
-      }
-      assert j == dest;
-    }
-
-    @Override
-    protected void merge(int lo, int pivot, int hi, int len1, int len2) {
-      if (len1 <= threshold) {
-        mergeWithExtraMemory(lo, pivot, hi, len1, len2);
-      } else {
-        // since this method recurses to run merge on smaller arrays, it will
-        // end up using mergeWithExtraMemory
-        super.merge(lo, pivot, hi, len1, len2);
-      }
+    public int compare(T o1, T o2) {
+      return o1.compareTo(o2);
     }
-
-  }
-
-  /** SorterTemplate with custom {@link Comparator} */
-  private static <T> SorterTemplate getSorter(final T[] a, final Comparator<? super T> comp) {
-    return new ArraySorterTemplate<T>(a) {
-
-      @Override
-      protected int compare(T a, T b) {
-        return comp.compare(a, b);
-      }
-
-    };
   }
 
-  /** Natural SorterTemplate */
-  private static <T extends Comparable<? super T>> SorterTemplate getSorter(final T[] a) {
-    return new ArraySorterTemplate<T>(a) {
-
-      @Override
-      protected int compare(T a, T b) {
-        return a.compareTo(b);
-      }
+  private static final Comparator<?> NATURAL_COMPARATOR = new NaturalComparator<>();
 
-    };
+  /** Get the natural {@link Comparator} for the provided object class. */
+  @SuppressWarnings("unchecked")
+  public static <T extends Comparable<? super T>> Comparator<T> naturalComparator() {
+    return (Comparator<T>) NATURAL_COMPARATOR;
   }
 
-  /** SorterTemplate with custom {@link Comparator} for merge-based sorts. */
-  private static <T> SorterTemplate getMergeSorter(final T[] a, final Comparator<? super T> comp) {
-    if (a.length < MERGE_EXTRA_MEMORY_THRESHOLD) {
-      return getSorter(a, comp);
-    } else {
-      return new ArrayMergeSorterTemplate<T>(a, MERGE_OVERHEAD_RATIO) {
-
-        @Override
-        protected int compare(T a, T b) {
-          return comp.compare(a, b);
-        }
-
-      };
-    }
+  /** Swap values stored in slots <code>i</code> and <code>j</code> */
+  public static <T> void swap(T[] arr, int i, int j) {
+    final T tmp = arr[i];
+    arr[i] = arr[j];
+    arr[j] = tmp;
   }
 
-  /** Natural SorterTemplate for merge-based sorts. */
-  private static <T extends Comparable<? super T>> SorterTemplate getMergeSorter(final T[] a) {
-    if (a.length < MERGE_EXTRA_MEMORY_THRESHOLD) {
-      return getSorter(a);
-    } else {
-      return new ArrayMergeSorterTemplate<T>(a, MERGE_OVERHEAD_RATIO) {
-
-        @Override
-        protected int compare(T a, T b) {
-          return a.compareTo(b);
-        }
-
-      };
-    }
-  }
-
-  // quickSorts (endindex is exclusive!):
+  // intro-sorts
   
   /**
-   * Sorts the given array slice using the {@link Comparator}. This method uses the quick sort
+   * Sorts the given array slice using the {@link Comparator}. This method uses the intro 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) {
+  public static <T> void introSort(T[] a, int fromIndex, int toIndex, Comparator<? super T> comp) {
     if (toIndex-fromIndex <= 1) return;
-    getSorter(a, comp).quickSort(fromIndex, toIndex-1);
+    new ArrayIntroSorter<>(a, comp).sort(fromIndex, toIndex);
   }
   
   /**
-   * Sorts the given array using the {@link Comparator}. This method uses the quick sort
+   * Sorts the given array using the {@link Comparator}. This method uses the intro 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);
+  public static <T> void introSort(T[] a, Comparator<? super T> comp) {
+    introSort(a, 0, a.length, comp);
   }
   
   /**
-   * Sorts the given array slice in natural order. This method uses the quick sort
+   * Sorts the given array slice in natural order. This method uses the intro 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) {
+  public static <T extends Comparable<? super T>> void introSort(T[] a, int fromIndex, int toIndex) {
     if (toIndex-fromIndex <= 1) return;
-    getSorter(a).quickSort(fromIndex, toIndex-1);
+    final Comparator<T> comp = naturalComparator();
+    introSort(a, fromIndex, toIndex, comp);
   }
   
   /**
-   * Sorts the given array in natural order. This method uses the quick sort
+   * Sorts the given array in natural order. This method uses the intro 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);
+  public static <T extends Comparable<? super T>> void introSort(T[] a) {
+    introSort(a, 0, a.length);
   }
 
-  // mergeSorts:
+  // tim sorts:
   
   /**
-   * 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) {
-    if (toIndex-fromIndex <= 1) return;
-    //System.out.println("SORT: " + (toIndex-fromIndex));
-    getMergeSorter(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) {
-    if (toIndex-fromIndex <= 1) return;
-    getMergeSorter(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);
-  }
-
-  // timSorts:
-
-  /**
-   * Sorts the given array slice using the {@link Comparator}. This method uses the TimSort
+   * Sorts the given array slice using the {@link Comparator}. This method uses the Tim sort
    * algorithm, but falls back to binary sort for small arrays.
    * @param fromIndex start index (inclusive)
    * @param toIndex end index (exclusive)
    */
   public static <T> void timSort(T[] a, int fromIndex, int toIndex, Comparator<? super T> comp) {
     if (toIndex-fromIndex <= 1) return;
-    getMergeSorter(a, comp).timSort(fromIndex, toIndex-1);
+    new ArrayTimSorter<>(a, comp, a.length / 64).sort(fromIndex, toIndex);
   }
   
   /**
-   * Sorts the given array using the {@link Comparator}. This method uses the TimSort
+   * Sorts the given array using the {@link Comparator}. This method uses the Tim sort
    * algorithm, but falls back to binary sort for small arrays.
    */
   public static <T> void timSort(T[] a, Comparator<? super T> comp) {
@@ -848,102 +691,23 @@ public final class ArrayUtil {
   }
   
   /**
-   * Sorts the given array slice in natural order. This method uses the TimSort
+   * Sorts the given array slice in natural order. This method uses the Tim sort
    * algorithm, but falls back to binary sort for small arrays.
    * @param fromIndex start index (inclusive)
    * @param toIndex end index (exclusive)
    */
   public static <T extends Comparable<? super T>> void timSort(T[] a, int fromIndex, int toIndex) {
     if (toIndex-fromIndex <= 1) return;
-    getMergeSorter(a).timSort(fromIndex, toIndex-1);
+    final Comparator<T> comp = naturalComparator();
+    timSort(a, fromIndex, toIndex, comp);
   }
   
   /**
-   * Sorts the given array in natural order. This method uses the TimSort
+   * Sorts the given array in natural order. This method uses the Tim sort
    * algorithm, but falls back to binary sort for small arrays.
    */
   public static <T extends Comparable<? super T>> void timSort(T[] a) {
     timSort(a, 0, a.length);
   }
 
-  // insertionSorts:
-  
-  /**
-   * Sorts the given array slice using the {@link Comparator}. This method uses the insertion sort
-   * algorithm. It is only recommended 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) {
-    if (toIndex-fromIndex <= 1) return;
-    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 recommended 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 recommended 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) {
-    if (toIndex-fromIndex <= 1) return;
-    getSorter(a).insertionSort(fromIndex, toIndex-1);
-  }
-  
-  /**
-   * Sorts the given array in natural order. This method uses the insertion sort
-   * algorithm. It is only recommended 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);
-  }
-
-  // binarySorts:
-
-  /**
-   * Sorts the given array slice using the {@link Comparator}. This method uses the binary sort
-   * algorithm. It is only recommended to use this algorithm for small arrays!
-   * @param fromIndex start index (inclusive)
-   * @param toIndex end index (exclusive)
-   */
-  public static <T> void binarySort(T[] a, int fromIndex, int toIndex, Comparator<? super T> comp) {
-    if (toIndex-fromIndex <= 1) return;
-    getSorter(a, comp).binarySort(fromIndex, toIndex-1);
-  }
-  
-  /**
-   * Sorts the given array using the {@link Comparator}. This method uses the binary sort
-   * algorithm. It is only recommended to use this algorithm for small arrays!
-   */
-  public static <T> void binarySort(T[] a, Comparator<? super T> comp) {
-    binarySort(a, 0, a.length, comp);
-  }
-  
-  /**
-   * Sorts the given array slice in natural order. This method uses the binary sort
-   * algorithm. It is only recommended to use this algorithm for small arrays!
-   * @param fromIndex start index (inclusive)
-   * @param toIndex end index (exclusive)
-   */
-  public static <T extends Comparable<? super T>> void binarySort(T[] a, int fromIndex, int toIndex) {
-    if (toIndex-fromIndex <= 1) return;
-    getSorter(a).binarySort(fromIndex, toIndex-1);
-  }
-  
-  /**
-   * Sorts the given array in natural order. This method uses the binary sort
-   * algorithm. It is only recommended to use this algorithm for small arrays!
-   */
-  public static <T extends Comparable<? super T>> void binarySort(T[] a) {
-    binarySort(a, 0, a.length);
-  }
-
-}
\ No newline at end of file
+}

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java?rev=1478785&r1=1478784&r2=1478785&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java Fri May  3 13:37:45 2013
@@ -163,7 +163,7 @@ public final class BytesRefHash {
    */
   public int[] sort(final Comparator<BytesRef> comp) {
     final int[] compact = compact();
-    new SorterTemplate() {
+    new IntroSorter() {
       @Override
       protected void swap(int i, int j) {
         final int o = compact[i];
@@ -197,7 +197,7 @@ public final class BytesRefHash {
       
       private final BytesRef pivot = new BytesRef(),
         scratch1 = new BytesRef(), scratch2 = new BytesRef();
-    }.quickSort(0, count - 1);
+    }.sort(0, count);
     return compact;
   }
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/CollectionUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/CollectionUtil.java?rev=1478785&r1=1478784&r2=1478785&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/CollectionUtil.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/CollectionUtil.java Fri May  3 13:37:45 2013
@@ -17,9 +17,6 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
-import static org.apache.lucene.util.ArrayUtil.MERGE_EXTRA_MEMORY_THRESHOLD;
-import static org.apache.lucene.util.ArrayUtil.MERGE_OVERHEAD_RATIO;
-
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
@@ -37,16 +34,22 @@ import java.util.RandomAccess;
 public final class CollectionUtil {
 
   private CollectionUtil() {} // no instance
+  private static final class ListIntroSorter<T> extends IntroSorter {
 
-  private static abstract class ListSorterTemplate<T> extends SorterTemplate {
-
-    protected final List<T> list;
+    T pivot;
+    final List<T> list;
+    final Comparator<? super T> comp;
 
-    ListSorterTemplate(List<T> list) {
+    ListIntroSorter(List<T> list, Comparator<? super T> comp) {
+      super();
       this.list = list;
+      this.comp = comp;
     }
 
-    protected abstract int compare(T a, T b);
+    @Override
+    protected void setPivot(int i) {
+      pivot = list.get(i);
+    }
 
     @Override
     protected void swap(int i, int j) {
@@ -55,257 +58,118 @@ public final class CollectionUtil {
 
     @Override
     protected int compare(int i, int j) {
-      return compare(list.get(i), list.get(j));
-    }
-
-    @Override
-    protected void setPivot(int i) {
-      pivot = list.get(i);
+      return comp.compare(list.get(i), list.get(j));
     }
 
     @Override
     protected int comparePivot(int j) {
-      return compare(pivot, list.get(j));
+      return comp.compare(pivot, list.get(j));
     }
 
-    private T pivot;
-
   }
 
-  // a template for merge-based sorts which uses extra memory to speed up merging
-  private static abstract class ListMergeSorterTemplate<T> extends ListSorterTemplate<T> {
+  private static final class ListTimSorter<T> extends TimSorter {
 
-    private final int threshold; // maximum length of a merge that can be made using extra memory
-    private final T[] tmp;
+    final List<T> list;
+    final Comparator<? super T> comp;
+    final T[] tmp;
 
-    ListMergeSorterTemplate(List<T> list, float overheadRatio) {
-      super(list);
-      this.threshold = (int) (list.size() * overheadRatio);
-      @SuppressWarnings("unchecked")
-      final T[] tmpBuf = (T[]) new Object[threshold];
-      this.tmp = tmpBuf;
-    }
-
-    private void mergeWithExtraMemory(int lo, int pivot, int hi, int len1, int len2) {
-      for (int i = 0; i < len1; ++i) {
-        tmp[i] = list.get(lo + i);
-      }
-      int i = 0, j = pivot, dest = lo;
-      while (i < len1 && j < hi) {
-        if (compare(tmp[i], list.get(j)) <= 0) {
-          list.set(dest++, tmp[i++]);
-        } else {
-          list.set(dest++, list.get(j++));
-        }
-      }
-      while (i < len1) {
-        list.set(dest++, tmp[i++]);
+    @SuppressWarnings("unchecked")
+    ListTimSorter(List<T> list, Comparator<? super T> comp, int maxTempSlots) {
+      super(maxTempSlots);
+      this.list = list;
+      this.comp = comp;
+      if (maxTempSlots > 0) {
+        this.tmp = (T[]) new Object[maxTempSlots];
+      } else {
+        this.tmp = null;
       }
-      assert j == dest;
     }
 
     @Override
-    protected void merge(int lo, int pivot, int hi, int len1, int len2) {
-      if (len1 <= threshold) {
-        mergeWithExtraMemory(lo, pivot, hi, len1, len2);
-      } else {
-        // since this method recurses to run merge on smaller arrays, it will
-        // end up using mergeWithExtraMemory
-        super.merge(lo, pivot, hi, len1, len2);
-      }
+    protected void swap(int i, int j) {
+      Collections.swap(list, i, j);
     }
 
-  }
-
-  /** 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 ListSorterTemplate<T>(list) {
-
-      @Override
-      protected int compare(T a, T b) {
-        return comp.compare(a, b);
-      }
+    @Override
+    protected void copy(int src, int dest) {
+      list.set(dest, list.get(src));
+    }
 
-    };
-  }
-  
-  /** 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 ListSorterTemplate<T>(list) {
-
-      @Override
-      protected int compare(T a, T b) {
-        return a.compareTo(b);
+    @Override
+    protected void save(int i, int len) {
+      for (int j = 0; j < len; ++j) {
+        tmp[j] = list.get(i + j);
       }
+    }
 
-    };
-  }
-
-  /** SorterTemplate with custom {@link Comparator} for merge-based sorts. */
-  private static <T> SorterTemplate getMergeSorter(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.");
-    if (list.size() < MERGE_EXTRA_MEMORY_THRESHOLD) {
-      return getSorter(list, comp);
-    } else {
-      return new ListMergeSorterTemplate<T>(list, MERGE_OVERHEAD_RATIO) {
-
-        @Override
-        protected int compare(T a, T b) {
-          return comp.compare(a, b);
-        }
+    @Override
+    protected void restore(int i, int j) {
+      list.set(j, tmp[i]);
+    }
 
-      };
+    @Override
+    protected int compare(int i, int j) {
+      return comp.compare(list.get(i), list.get(j));
     }
-  }
-  
-  /** Natural SorterTemplate for merge-based sorts. */
-  private static <T extends Comparable<? super T>> SorterTemplate getMergeSorter(final List<T> list) {
-    if (!(list instanceof RandomAccess))
-      throw new IllegalArgumentException("CollectionUtil can only sort random access lists in-place.");
-    if (list.size() < MERGE_EXTRA_MEMORY_THRESHOLD) {
-      return getSorter(list);
-    } else {
-      return new ListMergeSorterTemplate<T>(list, MERGE_OVERHEAD_RATIO) {
-
-        @Override
-        protected int compare(T a, T b) {
-          return a.compareTo(b);
-        }
 
-      };
+    @Override
+    protected int compareSaved(int i, int j) {
+      return comp.compare(tmp[i], list.get(j));
     }
-  }
 
-  /**
-   * 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) {
-    final int size = list.size();
-    if (size <= 1) return;
-    getSorter(list, comp).quickSort(0, 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) {
-    final int size = list.size();
-    if (size <= 1) return;
-    getSorter(list).quickSort(0, 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
+   * The list must implement {@link RandomAccess}. This method uses the intro 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) {
+  public static <T> void introSort(List<T> list, Comparator<? super T> comp) {
     final int size = list.size();
     if (size <= 1) return;
-    getMergeSorter(list, comp).mergeSort(0, size-1);
+    new ListIntroSorter<>(list, comp).sort(0, size);
   }
   
   /**
    * Sorts the given random access {@link List} in natural order.
-   * The list must implement {@link RandomAccess}. This method uses the merge sort
+   * The list must implement {@link RandomAccess}. This method uses the intro 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) {
+  public static <T extends Comparable<? super T>> void introSort(List<T> list) {
     final int size = list.size();
     if (size <= 1) return;
-    getMergeSorter(list).mergeSort(0, size-1);
+    final Comparator<T> comp = ArrayUtil.naturalComparator();
+    introSort(list, comp);
   }
 
-  // timSorts:
+  // Tim sorts:
   
   /**
    * Sorts the given random access {@link List} using the {@link Comparator}.
-   * The list must implement {@link RandomAccess}. This method uses the TimSort
+   * The list must implement {@link RandomAccess}. This method uses the Tim sort
    * algorithm, but falls back to binary sort for small lists.
    * @throws IllegalArgumentException if list is e.g. a linked list without random access.
    */
   public static <T> void timSort(List<T> list, Comparator<? super T> comp) {
     final int size = list.size();
     if (size <= 1) return;
-    getMergeSorter(list, comp).timSort(0, size-1);
+    new ListTimSorter<>(list, comp, list.size() / 64).sort(0, size);
   }
   
   /**
    * Sorts the given random access {@link List} in natural order.
-   * The list must implement {@link RandomAccess}. This method uses the TimSort
+   * The list must implement {@link RandomAccess}. This method uses the Tim sort
    * algorithm, but falls back to binary 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 timSort(List<T> list) {
     final int size = list.size();
     if (size <= 1) return;
-    getMergeSorter(list).timSort(0, 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 recommended 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) {
-    final int size = list.size();
-    if (size <= 1) return;
-    getSorter(list, comp).insertionSort(0, 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 recommended 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) {
-    final int size = list.size();
-    if (size <= 1) return;
-    getSorter(list).insertionSort(0, size-1);
+    final Comparator<T> comp = ArrayUtil.naturalComparator();
+    timSort(list, comp);
   }
 
-  // binarySorts:
-  
-  /**
-   * Sorts the given random access {@link List} using the {@link Comparator}.
-   * The list must implement {@link RandomAccess}. This method uses the binary sort
-   * algorithm. It is only recommended to use this algorithm for small lists!
-   * @throws IllegalArgumentException if list is e.g. a linked list without random access.
-   */
-  public static <T> void binarySort(List<T> list, Comparator<? super T> comp) {
-    final int size = list.size();
-    if (size <= 1) return;
-    getSorter(list, comp).binarySort(0, 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 recommended to use this algorithm for small lists!
-   * @throws IllegalArgumentException if list is e.g. a linked list without random access.
-   */
-  public static <T extends Comparable<? super T>> void binarySort(List<T> list) {
-    final int size = list.size();
-    if (size <= 1) return;
-    getSorter(list).binarySort(0, size-1);
-  }
-}
\ No newline at end of file
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/InPlaceMergeSorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/InPlaceMergeSorter.java?rev=1478785&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/InPlaceMergeSorter.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/InPlaceMergeSorter.java Fri May  3 13:37:45 2013
@@ -0,0 +1,46 @@
+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.
+ */
+
+/** {@link Sorter} implementation based on the merge-sort algorithm that merges
+ *  in place (no extra memory will be allocated). Small arrays are sorted with
+ *  insertion sort.
+ *  @lucene.internal */
+public abstract class InPlaceMergeSorter extends Sorter {
+
+  /** Create a new {@link InPlaceMergeSorter} */
+  public InPlaceMergeSorter() {}
+
+  @Override
+  public final void sort(int from, int to) {
+    checkRange(from, to);
+    mergeSort(from, to);
+  }
+
+  void mergeSort(int from, int to) {
+    if (to - from < THRESHOLD) {
+      insertionSort(from, to);
+    } else {
+      final int mid = (from + to) >>> 1;
+      mergeSort(from, mid);
+      mergeSort(mid, to);
+      mergeInPlace(from, mid, to);
+    }
+  }
+
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/IntroSorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/IntroSorter.java?rev=1478785&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/IntroSorter.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/IntroSorter.java Fri May  3 13:37:45 2013
@@ -0,0 +1,98 @@
+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.
+ */
+
+/**
+ * {@link Sorter} implementation based on a variant of the quicksort algorithm
+ * called <a href="http://en.wikipedia.org/wiki/Introsort">introsort</a>: when
+ * the recursion level exceeds the log of the length of the array to sort, it
+ * falls back to heapsort. This prevents quicksort from running into its
+ * worst-case quadratic runtime. Small arrays are sorted with
+ * insertion sort.
+ * @lucene.internal
+ */
+public abstract class IntroSorter extends Sorter {
+
+  static int ceilLog2(int n) {
+    return Integer.SIZE - Integer.numberOfLeadingZeros(n - 1);
+  }
+
+  /** Create a new {@link IntroSorter}. */
+  public IntroSorter() {}
+
+  @Override
+  public final void sort(int from, int to) {
+    checkRange(from, to);
+    quicksort(from, to, ceilLog2(to - from));
+  }
+
+  void quicksort(int from, int to, int maxDepth) {
+    if (to - from < THRESHOLD) {
+      insertionSort(from, to);
+      return;
+    } else if (--maxDepth < 0) {
+      heapSort(from, to);
+      return;
+    }
+
+    final int mid = (from + to) >>> 1;
+
+    if (compare(from, mid) > 0) {
+      swap(from, mid);
+    }
+
+    if (compare(mid, to - 1) > 0) {
+      swap(mid, to - 1);
+      if (compare(from, mid) > 0) {
+        swap(from, mid);
+      }
+    }
+
+    int left = from + 1;
+    int right = to - 2;
+
+    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(from, left + 1, maxDepth);
+    quicksort(left + 1, to, maxDepth);
+  }
+
+  /** Save the value at slot <code>i</code> so that it can later be used as a
+   * pivot, see {@link #comparePivot(int)}. */
+  protected abstract void setPivot(int i);
+
+  /** Compare the pivot with the slot at <code>j</code>, similarly to
+   *  {@link #compare(int, int) compare(i, j)}. */
+  protected abstract int comparePivot(int j);
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/Sorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/Sorter.java?rev=1478785&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/Sorter.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/Sorter.java Fri May  3 13:37:45 2013
@@ -0,0 +1,248 @@
+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;
+
+/** Base class for sorting algorithms implementations.
+ * @lucene.internal */
+public abstract class Sorter {
+
+  static final int THRESHOLD = 20;
+
+  /** Sole constructor, used for inheritance. */
+  protected Sorter() {}
+
+  /** Compare entries found in slots <code>i</code> and <code>j</code>.
+   *  The contract for the returned value is the same as
+   *  {@link Comparator#compare(Object, Object)}. */
+  protected abstract int compare(int i, int j);
+
+  /** Swap values at slots <code>i</code> and <code>j</code>. */
+  protected abstract void swap(int i, int j);
+
+  /** Sort the slice which starts at <code>from</code> (inclusive) and ends at
+   *  <code>to</code> (exclusive). */
+  public abstract void sort(int from, int to);
+
+  void checkRange(int from, int to) {
+    if (to < from) {
+      throw new IllegalArgumentException("'to' must be >= 'from', got from=" + from + " and to=" + to);
+    }
+  }
+
+  void mergeInPlace(int from, int mid, int to) {
+    if (from == mid || mid == to || compare(mid - 1, mid) <= 0) {
+      return;
+    } else if (to - from == 2) {
+      swap(mid - 1, mid);
+      return;
+    }
+    while (compare(from, mid) <= 0) {
+      ++from;
+    }
+    while (compare(mid - 1, to - 1) <= 0) {
+      --to;
+    }
+    int first_cut, second_cut;
+    int len11, len22;
+    if (mid - from > to - mid) {
+      len11 = (mid - from) >>> 1;
+      first_cut = from + len11;
+      second_cut = lower(mid, to, first_cut);
+      len22 = second_cut - mid;
+    } else {
+      len22 = (to - mid) >>> 1;
+      second_cut = mid + len22;
+      first_cut = upper(from, mid, second_cut);
+      len11 = first_cut - from;
+    }
+    rotate( first_cut, mid, second_cut);
+    final int new_mid = first_cut + len22;
+    mergeInPlace(from, first_cut, new_mid);
+    mergeInPlace(new_mid, second_cut, to);
+  }
+
+  int lower(int from, int to, int val) {
+    int len = to - from;
+    while (len > 0) {
+      final int half = len >>> 1;
+      final int mid = from + half;
+      if (compare(mid, val) < 0) {
+        from = mid + 1;
+        len = len - half -1;
+      } else {
+        len = half;
+      }
+    }
+    return from;
+  }
+
+  int upper(int from, int to, int val) {
+    int len = to - from;
+    while (len > 0) {
+      final int half = len >>> 1;
+      final int mid = from + half;
+      if (compare(val, mid) < 0) {
+        len = half;
+      } else {
+        from = mid + 1;
+        len = len - half -1;
+      }
+    }
+    return from;
+  }
+
+  // faster than lower when val is at the end of [from:to[
+  int lower2(int from, int to, int val) {
+    int f = to - 1, t = to;
+    while (f > from) {
+      if (compare(f, val) < 0) {
+        return lower(f, t, val);
+      }
+      final int delta = t - f;
+      t = f;
+      f -= delta << 1;
+    }
+    return lower(from, t, val);
+  }
+
+  // faster than upper when val is at the beginning of [from:to[
+  int upper2(int from, int to, int val) {
+    int f = from, t = f + 1;
+    while (t < to) {
+      if (compare(t, val) > 0) {
+        return upper(f, t, val);
+      }
+      final int delta = t - f;
+      f = t;
+      t += delta << 1;
+    }
+    return upper(f, to, val);
+  }
+
+  final void reverse(int from, int to) {
+    for (--to; from < to; ++from, --to) {
+      swap(from, to);
+    }
+  }
+
+  void rotate(int lo, int mid, int hi) {
+    if (mid - lo == hi - mid) {
+      // happens rarely but saves n/2 swaps
+      while (mid < hi) {
+        swap(lo++, mid++);
+      }
+    } else {
+      reverse(lo, mid);
+      reverse(mid, hi);
+      reverse(lo, hi);
+    }
+  }
+
+  void insertionSort(int from, int to) {
+    for (int i = from + 1; i < to; ++i) {
+      for (int j = i; j > from; --j) {
+        if (compare(j - 1, j) > 0) {
+          swap(j - 1, j);
+        } else {
+          break;
+        }
+      }
+    }
+  }
+
+  void binarySort(int from, int to) {
+    binarySort(from, to, from + 1);
+  }
+
+  void binarySort(int from, int to, int i) {
+    for ( ; i < to; ++i) {
+      int l = from;
+      int h = i - 1;
+      while (l <= h) {
+        final int mid = (l + h) >>> 1;
+        final int cmp = compare(i, mid);
+        if (cmp < 0) {
+          h = mid - 1;
+        } else {
+          l = mid + 1;
+        }
+      }
+      switch (i - l) {
+      case 2:
+        swap(l + 1, l + 2);
+      case 1:
+        swap(l, l + 1);
+      case 0:
+        break;
+      default:
+        for (int j = i; j > l; --j) {
+          swap(j - 1, j);
+        }
+        break;
+      }
+    }
+  }
+
+  void heapSort(int from, int to) {
+    if (to - from <= 1) {
+      return;
+    }
+    heapify(from, to);
+    for (int end = to - 1; end > from; --end) {
+      swap(from, end);
+      siftDown(from, from, end);
+    }
+  }
+
+  void heapify(int from, int to) {
+    for (int i = heapParent(from, to - 1); i >= from; --i) {
+      siftDown(i, from, to);
+    }
+  }
+
+  void siftDown(int i, int from, int to) {
+    for (int leftChild = heapChild(from, i); leftChild < to; leftChild = heapChild(from, i)) {
+      final int rightChild = leftChild + 1;
+      if (compare(i, leftChild) < 0) {
+        if (rightChild < to && compare(leftChild, rightChild) < 0) {
+          swap(i, rightChild);
+          i = rightChild;
+        } else {
+          swap(i, leftChild);
+          i = leftChild;
+        }
+      } else if (rightChild < to && compare(i, rightChild) < 0) {
+        swap(i, rightChild);
+        i = rightChild;
+      } else {
+        break;
+      }
+    }
+  }
+
+  static int heapParent(int from, int i) {
+    return ((i - 1 - from) >>> 1) + from;
+  }
+
+  static int heapChild(int from, int i) {
+    return ((i - from) << 1) + 1 + from;
+  }
+
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/TimSorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/TimSorter.java?rev=1478785&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/TimSorter.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/TimSorter.java Fri May  3 13:37:45 2013
@@ -0,0 +1,373 @@
+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;
+
+/**
+ * {@link Sorter} implementation based on the
+ * <a href="http://svn.python.org/projects/python/trunk/Objects/listsort.txt">TimSort</a>
+ * algorithm.
+ * <p>This implementation is especially good at sorting partially-sorted
+ * arrays and sorts small arrays with binary sort.
+ * <p><b>NOTE</b>:There are a few differences with the original implementation:<ul>
+ * <li><a name="maxTempSlots"/>The extra amount of memory to perform merges is
+ * configurable. This allows small merges to be very fast while large merges
+ * will be performed in-place (slightly slower). You can make sure that the
+ * fast merge routine will always be used by having <code>maxTempSlots</code>
+ * equal to half of the length of the slice of data to sort.
+ * <li>Only the fast merge routine can gallop (the one that doesn't run
+ * in-place) and it only gallops on the longest slice.
+ * </ul>
+ * @lucene.internal
+ */
+public abstract class TimSorter extends Sorter {
+
+  static final int MINRUN = 32;
+  static final int THRESHOLD = 64;
+  static final int STACKSIZE = 40; // depends on MINRUN
+  static final int MIN_GALLOP = 7;
+
+  final int maxTempSlots;
+  int minRun;
+  int to;
+  int stackSize;
+  int[] runEnds;
+
+  /**
+   * Create a new {@link TimSorter}.
+   * @param maxTempSlots the <a href="#maxTempSlots">maximum amount of extra memory to run merges</a>
+   */
+  protected TimSorter(int maxTempSlots) {
+    super();
+    runEnds = new int[1 + STACKSIZE];
+    this.maxTempSlots = maxTempSlots;
+  }
+
+  /** Minimum run length for an array of length <code>length</code>. */
+  static int minRun(int length) {
+    assert length >= MINRUN;
+    int n = length;
+    int r = 0;
+    while (n >= 64) {
+      r |= n & 1;
+      n >>>= 1;
+    }
+    final int minRun = n + r;
+    assert minRun >= MINRUN && minRun <= THRESHOLD;
+    return minRun;
+  }
+
+  int runLen(int i) {
+    final int off = stackSize - i;
+    return runEnds[off] - runEnds[off - 1];
+  }
+
+  int runBase(int i) {
+    return runEnds[stackSize - i - 1];
+  }
+
+  int runEnd(int i) {
+    return runEnds[stackSize - i];
+  }
+
+  void setRunEnd(int i, int runEnd) {
+    runEnds[stackSize - i] = runEnd;
+  }
+
+  void pushRunLen(int len) {
+    runEnds[stackSize + 1] = runEnds[stackSize] + len;
+    ++stackSize;
+  }
+
+  /** Compute the length of the next run, make the run sorted and return its
+   *  length. */
+  int nextRun() {
+    final int runBase = runEnd(0);
+    assert runBase < to;
+    if (runBase == to - 1) {
+      return 1;
+    }
+    int o = runBase + 2;
+    if (compare(runBase, runBase+1) > 0) {
+      // run must be strictly descending
+      while (o < to && compare(o - 1, o) > 0) {
+        ++o;
+      }
+      reverse(runBase, o);
+    } else {
+      // run must be non-descending
+      while (o < to && compare(o - 1, o) <= 0) {
+        ++o;
+      }
+    }
+    final int runHi = Math.max(o, Math.min(to, runBase + minRun));
+    binarySort(runBase, runHi, o);
+    return runHi - runBase;
+  }
+
+  void ensureInvariants() {
+    while (stackSize > 1) {
+      final int runLen0 = runLen(0);
+      final int runLen1 = runLen(1);
+
+      if (stackSize > 2) {
+        final int runLen2 = runLen(2);
+
+        if (runLen2 <= runLen1 + runLen0) {
+          // merge the smaller of 0 and 2 with 1
+          if (runLen2 < runLen0) {
+            mergeAt(1);
+          } else {
+            mergeAt(0);
+          }
+          continue;
+        }
+      }
+
+      if (runLen1 <= runLen0) {
+        mergeAt(0);
+        continue;
+      }
+
+      break;
+    }
+  }
+
+  void exhaustStack() {
+    while (stackSize > 1) {
+      mergeAt(0);
+    }
+  }
+
+  void reset(int from, int to) {
+    stackSize = 0;
+    Arrays.fill(runEnds, 0);
+    runEnds[0] = from;
+    this.to = to;
+    final int length = to - from;
+    this.minRun = length <= THRESHOLD ? length : minRun(length);
+  }
+
+  void mergeAt(int n) {
+    assert stackSize >= 2;
+    merge(runBase(n + 1), runBase(n), runEnd(n));
+    for (int j = n + 1; j > 0; --j) {
+      setRunEnd(j, runEnd(j-1));
+    }
+    --stackSize;
+  }
+
+  void merge(int lo, int mid, int hi) {
+    if (compare(mid - 1, mid) <= 0) {
+      return;
+    }
+    lo = upper2(lo, mid, mid);
+    hi = lower2(mid, hi, mid - 1);
+
+    if (hi - mid <= mid - lo && hi - mid <= maxTempSlots) {
+      mergeHi(lo, mid, hi);
+    } else if (mid - lo <= maxTempSlots) {
+      mergeLo(lo, mid, hi);
+    } else {
+      mergeInPlace(lo, mid, hi);
+    }
+  }
+
+  @Override
+  public void sort(int from, int to) {
+    checkRange(from, to);
+    if (to - from <= 1) {
+      return;
+    }
+    reset(from, to);
+    do {
+      ensureInvariants();
+      pushRunLen(nextRun());
+    } while (runEnd(0) < to);
+    exhaustStack();
+    assert runEnd(0) == to;
+  }
+
+  @Override
+  void rotate(int lo, int mid, int hi) {
+    int len1 = mid - lo;
+    int len2 = hi - mid;
+    if (len1 == len2) {
+      while (mid < hi) {
+        swap(lo++, mid++);
+      }
+    } else if (len2 < len1 && len2 <= maxTempSlots) {
+      save(mid, len2);
+      for (int i = lo + len1 - 1, j = hi - 1; i >= lo; --i, --j) {
+        copy(i, j);
+      }
+      for (int i = 0, j = lo; i < len2; ++i, ++j) {
+        restore(i, j);
+      }
+    } else if (len1 <= maxTempSlots) {
+      save(lo, len1);
+      for (int i = mid, j = lo; i < hi; ++i, ++j) {
+        copy(i, j);
+      }
+      for (int i = 0, j = lo + len2; j < hi; ++i, ++j) {
+        restore(i, j);
+      }
+    } else {
+      reverse(lo, mid);
+      reverse(mid, hi);
+      reverse(lo, hi);
+    }
+  }
+
+  void mergeLo(int lo, int mid, int hi) {
+    assert compare(lo, mid) > 0;
+    int len1 = mid - lo;
+    save(lo, len1);
+    copy(mid, lo);
+    int i = 0, j = mid + 1, dest = lo + 1;
+    outer: for (;;) {
+      for (int count = 0; count < MIN_GALLOP; ) {
+        if (i >= len1 || j >= hi) {
+          break outer;
+        } else if (compareSaved(i, j) <= 0) {
+          restore(i++, dest++);
+          count = 0;
+        } else {
+          copy(j++, dest++);
+          ++count;
+        }
+      }
+      // galloping...
+      int next = lowerSaved3(j, hi, i);
+      for (; j < next; ++dest) {
+        copy(j++, dest);
+      }
+      restore(i++, dest++);
+    }
+    for (; i < len1; ++dest) {
+      restore(i++, dest);
+    }
+    assert j == dest;
+  }
+
+  void mergeHi(int lo, int mid, int hi) {
+    assert compare(mid - 1, hi - 1) > 0;
+    int len2 = hi - mid;
+    save(mid, len2);
+    copy(mid - 1, hi - 1);
+    int i = mid - 2, j = len2 - 1, dest = hi - 2;
+    outer: for (;;) {
+      for (int count = 0; count < MIN_GALLOP; ) {
+        if (i < lo || j < 0) {
+          break outer;
+        } else if (compareSaved(j, i) >= 0) {
+          restore(j--, dest--);
+          count = 0;
+        } else {
+          copy(i--, dest--);
+          ++count;
+        }
+      }
+      // galloping
+      int next = upperSaved3(lo, i + 1, j);
+      while (i >= next) {
+        copy(i--, dest--);
+      }
+      restore(j--, dest--);
+    }
+    for (; j >= 0; --dest) {
+      restore(j--, dest);
+    }
+    assert i == dest;
+  }
+
+  int lowerSaved(int from, int to, int val) {
+    int len = to - from;
+    while (len > 0) {
+      final int half = len >>> 1;
+      final int mid = from + half;
+      if (compareSaved(val, mid) > 0) {
+        from = mid + 1;
+        len = len - half -1;
+      } else {
+        len = half;
+      }
+    }
+    return from;
+  }
+
+  int upperSaved(int from, int to, int val) {
+    int len = to - from;
+    while (len > 0) {
+      final int half = len >>> 1;
+      final int mid = from + half;
+      if (compareSaved(val, mid) < 0) {
+        len = half;
+      } else {
+        from = mid + 1;
+        len = len - half -1;
+      }
+    }
+    return from;
+  }
+
+  // faster than lowerSaved when val is at the beginning of [from:to[
+  int lowerSaved3(int from, int to, int val) {
+    int f = from, t = f + 1;
+    while (t < to) {
+      if (compareSaved(val, t) <= 0) {
+        return lowerSaved(f, t, val);
+      }
+      int delta = t - f;
+      f = t;
+      t += delta << 1;
+    }
+    return lowerSaved(f, to, val);
+  }
+
+  //faster than upperSaved when val is at the end of [from:to[
+  int upperSaved3(int from, int to, int val) {
+    int f = to - 1, t = to;
+    while (f > from) {
+      if (compareSaved(val, f) >= 0) {
+        return upperSaved(f, t, val);
+      }
+      final int delta = t - f;
+      t = f;
+      f -= delta << 1;
+    }
+    return upperSaved(from, t, val);
+  }
+
+  /** Copy data from slot <code>src</code> to slot <code>dest</code>. */
+  protected abstract void copy(int src, int dest);
+
+  /** Save all elements between slots <code>i</code> and <code>i+len</code>
+   *  into the temporary storage. */
+  protected abstract void save(int i, int len);
+
+  /** Restore element <code>j</code> from the temporary storage into slot <code>i</code>. */
+  protected abstract void restore(int i, int j);
+
+  /** Compare element <code>i</code> from the temporary storage with element
+   *  <code>j</code> from the slice to sort, similarly to
+   *  {@link #compare(int, int)}. */
+  protected abstract int compareSaved(int i, int j);
+
+}

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/BasicOperations.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/BasicOperations.java?rev=1478785&r1=1478784&r2=1478785&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/BasicOperations.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/BasicOperations.java Fri May  3 13:37:45 2013
@@ -557,8 +557,8 @@ final public class BasicOperations {
     }
 
     public void sort() {
-      // mergesort seems to perform better on already sorted arrays:
-      if (count > 1) ArrayUtil.mergeSort(points, 0, count);
+      // Tim sort performs well on already sorted arrays:
+      if (count > 1) ArrayUtil.timSort(points, 0, count);
     }
 
     public void add(Transition t) {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/State.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/State.java?rev=1478785&r1=1478784&r2=1478785&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/State.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/State.java Fri May  3 13:37:45 2013
@@ -239,7 +239,7 @@ public class State implements Comparable
   /** Sorts transitions array in-place. */
   public void sortTransitions(Comparator<Transition> comparator) {
     // mergesort seems to perform better on already sorted arrays:
-    if (numTransitions > 1) ArrayUtil.mergeSort(transitionsArray, 0, numTransitions, comparator);
+    if (numTransitions > 1) ArrayUtil.timSort(transitionsArray, 0, numTransitions, comparator);
   }
   
   /**

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/BaseSortTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/BaseSortTestCase.java?rev=1478785&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/BaseSortTestCase.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/BaseSortTestCase.java Fri May  3 13:37:45 2013
@@ -0,0 +1,173 @@
+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;
+
+public abstract class BaseSortTestCase extends LuceneTestCase {
+
+  public static class Entry implements java.lang.Comparable<Entry> {
+
+    public final int value;
+    public final int ord;
+
+    public Entry(int value, int ord) {
+      this.value = value;
+      this.ord = ord;
+    }
+
+    @Override
+    public int compareTo(Entry other) {
+      return value < other.value ? -1 : value == other.value ? 0 : 1;
+    }
+
+  }
+
+  private final boolean stable;
+
+  public BaseSortTestCase(boolean stable) {
+    this.stable = stable;
+  }
+
+  public abstract Sorter newSorter(Entry[] arr);
+
+  public void assertSorted(Entry[] original, Entry[] sorted) {
+    assertEquals(original.length, sorted.length);
+    Entry[] actuallySorted = Arrays.copyOf(original, original.length);
+    Arrays.sort(actuallySorted);
+    for (int i = 0; i < original.length; ++i) {
+      assertEquals(actuallySorted[i].value, sorted[i].value);
+      if (stable) {
+        assertEquals(actuallySorted[i].ord, sorted[i].ord);
+      }
+    }
+  }
+
+  public void test(Entry[] arr) {
+    final int o = random().nextInt(1000);
+    final Entry[] toSort = new Entry[o + arr.length + random().nextInt(3)];
+    System.arraycopy(arr, 0, toSort, o, arr.length);
+    final Sorter sorter = newSorter(toSort);
+    sorter.sort(o, o + arr.length);
+    assertSorted(arr, Arrays.copyOfRange(toSort, o, o + arr.length));
+  }
+
+  enum Strategy {
+    RANDOM {
+      @Override
+      public void set(Entry[] arr, int i) {
+        arr[i] = new Entry(random().nextInt(), i);
+      }
+    },
+    RANDOM_LOW_CARDINALITY {
+      @Override
+      public void set(Entry[] arr, int i) {
+        arr[i] = new Entry(random().nextInt(6), i);
+      }
+    },
+    ASCENDING {
+      @Override
+      public void set(Entry[] arr, int i) {
+        arr[i] = i == 0
+            ? new Entry(random().nextInt(6), 0)
+            : new Entry(arr[i - 1].value + random().nextInt(6), i);
+      }
+    },
+    DESCENDING {
+      @Override
+      public void set(Entry[] arr, int i) {
+        arr[i] = i == 0
+            ? new Entry(random().nextInt(6), 0)
+            : new Entry(arr[i - 1].value - random().nextInt(6), i);
+      }
+    },
+    STRICTLY_DESCENDING {
+      @Override
+      public void set(Entry[] arr, int i) {
+        arr[i] = i == 0
+            ? new Entry(random().nextInt(6), 0)
+            : new Entry(arr[i - 1].value - _TestUtil.nextInt(random(), 1, 5), i);
+      }
+    },
+    ASCENDING_SEQUENCES {
+      @Override
+      public void set(Entry[] arr, int i) {
+        arr[i] = i == 0
+            ? new Entry(random().nextInt(6), 0)
+            : new Entry(rarely() ? random().nextInt(1000) : arr[i - 1].value + random().nextInt(6), i);
+      }
+    },
+    MOSTLY_ASCENDING {
+      @Override
+      public void set(Entry[] arr, int i) {
+        arr[i] = i == 0
+            ? new Entry(random().nextInt(6), 0)
+            : new Entry(arr[i - 1].value + _TestUtil.nextInt(random(), -8, 10), i);
+      }
+    };
+    public abstract void set(Entry[] arr, int i);
+  }
+
+  public void test(Strategy strategy, int length) {
+    final Entry[] arr = new Entry[length];
+    for (int i = 0; i < arr.length; ++i) {
+      strategy.set(arr, i);
+    }
+    test(arr);
+  }
+
+  public void test(Strategy strategy) {
+    test(strategy, random().nextInt(20000));
+  }
+
+  public void testEmpty() {
+    test(new Entry[0]);
+  }
+
+  public void testOne() {
+    test(Strategy.RANDOM, 1);
+  }
+
+  public void testTwo() {
+    test(Strategy.RANDOM_LOW_CARDINALITY, 2);
+  }
+
+  public void testRandom() {
+    test(Strategy.RANDOM);
+  }
+
+  public void testRandomLowCardinality() {
+    test(Strategy.RANDOM_LOW_CARDINALITY);
+  }
+
+  public void testAscending() {
+    test(Strategy.ASCENDING);
+  }
+
+  public void testAscendingSequences() {
+    test(Strategy.ASCENDING_SEQUENCES);
+  }
+
+  public void testDescending() {
+    test(Strategy.DESCENDING);
+  }
+
+  public void testStrictlyDescending() {
+    test(Strategy.STRICTLY_DESCENDING);
+  }
+}

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestArrayUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestArrayUtil.java?rev=1478785&r1=1478784&r2=1478785&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestArrayUtil.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestArrayUtil.java Fri May  3 13:37:45 2013
@@ -128,21 +128,21 @@ public class TestArrayUtil extends Lucen
     return a;
   }
   
-  public void testQuickSort() {
+  public void testIntroSort() {
     int num = atLeast(50);
     for (int i = 0; i < num; i++) {
       Integer[] a1 = createRandomArray(2000), a2 = a1.clone();
-      ArrayUtil.quickSort(a1);
+      ArrayUtil.introSort(a1);
       Arrays.sort(a2);
       assertArrayEquals(a2, a1);
       
       a1 = createRandomArray(2000);
       a2 = a1.clone();
-      ArrayUtil.quickSort(a1, Collections.reverseOrder());
+      ArrayUtil.introSort(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);
+      ArrayUtil.introSort(a1);
       Arrays.sort(a2);
       assertArrayEquals(a2, a1);
     }
@@ -158,38 +158,18 @@ public class TestArrayUtil extends Lucen
   }
   
   // This is a test for LUCENE-3054 (which fails without the merge sort fall back with stack overflow in most cases)
-  public void testQuickToMergeSortFallback() {
+  public void testQuickToHeapSortFallback() {
     int num = atLeast(50);
     for (int i = 0; i < num; i++) {
       Integer[] a1 = createSparseRandomArray(40000), a2 = a1.clone();
-      ArrayUtil.quickSort(a1);
+      ArrayUtil.introSort(a1);
       Arrays.sort(a2);
       assertArrayEquals(a2, a1);
     }
   }
   
-  public void testMergeSort() {
-    int num = atLeast(50);
-    for (int i = 0; i < num; i++) {
-      Integer[] a1 = createRandomArray(2000), a2 = a1.clone();
-      ArrayUtil.mergeSort(a1);
-      Arrays.sort(a2);
-      assertArrayEquals(a2, a1);
-      
-      a1 = createRandomArray(2000);
-      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 testTimSort() {
-    int num = atLeast(65);
+    int num = atLeast(50);
     for (int i = 0; i < num; i++) {
       Integer[] a1 = createRandomArray(2000), a2 = a1.clone();
       ArrayUtil.timSort(a1);
@@ -207,44 +187,6 @@ public class TestArrayUtil extends Lucen
       assertArrayEquals(a2, a1);
     }
   }
-
-  public void testInsertionSort() {
-    for (int i = 0, c = atLeast(500); 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);
-    }
-  }
-  
-  public void testBinarySort() {
-    for (int i = 0, c = atLeast(500); i < c; i++) {
-      Integer[] a1 = createRandomArray(30), a2 = a1.clone();
-      ArrayUtil.binarySort(a1);
-      Arrays.sort(a2);
-      assertArrayEquals(a2, a1);
-      
-      a1 = createRandomArray(30);
-      a2 = a1.clone();
-      ArrayUtil.binarySort(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.binarySort(a1);
-      Arrays.sort(a2);
-      assertArrayEquals(a2, a1);
-    }
-  }
   
   static class Item implements Comparable<Item> {
     final int val, order;
@@ -279,7 +221,7 @@ public class TestArrayUtil extends Lucen
     
     if (VERBOSE) System.out.println("Before: " + Arrays.toString(items));
     // if you replace this with ArrayUtil.quickSort(), test should fail:
-    ArrayUtil.mergeSort(items);
+    ArrayUtil.timSort(items);
     if (VERBOSE) System.out.println("Sorted: " + Arrays.toString(items));
     
     Item last = items[0];
@@ -326,16 +268,10 @@ public class TestArrayUtil extends Lucen
   // should produce no exceptions
   public void testEmptyArraySort() {
     Integer[] a = new Integer[0];
-    ArrayUtil.quickSort(a);
-    ArrayUtil.mergeSort(a);
-    ArrayUtil.insertionSort(a);
-    ArrayUtil.binarySort(a);
+    ArrayUtil.introSort(a);
     ArrayUtil.timSort(a);
-    ArrayUtil.quickSort(a, Collections.reverseOrder());
-    ArrayUtil.mergeSort(a, Collections.reverseOrder());
+    ArrayUtil.introSort(a, Collections.reverseOrder());
     ArrayUtil.timSort(a, Collections.reverseOrder());
-    ArrayUtil.insertionSort(a, Collections.reverseOrder());
-    ArrayUtil.binarySort(a, Collections.reverseOrder());
   }
   
 }