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 2015/07/09 10:39:01 UTC

svn commit: r1690026 - in /lucene/dev/trunk/lucene: ./ core/src/java/org/apache/lucene/search/ core/src/java/org/apache/lucene/util/ core/src/test/org/apache/lucene/util/ join/src/java/org/apache/lucene/search/join/ misc/src/test/org/apache/lucene/inde...

Author: jpountz
Date: Thu Jul  9 08:39:00 2015
New Revision: 1690026

URL: http://svn.apache.org/r1690026
Log:
LUCENE-6645: Optimized DocIdSet building for the "many small postings lists" case.

Added:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/IntArrayDocIdSet.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/LSBRadixSorter.java   (with props)
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestIntArrayDocIdSet.java   (with props)
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestLSBRadixSorter.java   (with props)
    lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/util/BitDocIdSetBuilder.java   (with props)
Removed:
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestBitDocIdSetBuilder.java
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BitDocIdSet.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BitSet.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java
    lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/QueryBitSetProducer.java
    lucene/dev/trunk/lucene/misc/src/test/org/apache/lucene/index/TestBlockJoinSorter.java
    lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java
    lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeReader.java
    lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java
    lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeFilter.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1690026&r1=1690025&r2=1690026&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Thu Jul  9 08:39:00 2015
@@ -319,6 +319,10 @@ Optimizations
   where an expired segments_N references non-existing files (Robert
   Muir, Mike McCandless)
 
+* LUCENE-6645: Optimized the way we merge postings lists in multi-term queries
+  and TermsQuery. This should especially help when there are lots of small
+  postings lists. (Adrien Grand, Mike McCandless)
+
 Build
 
 * LUCENE-6518: Don't report false thread leaks from IBM J9

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java?rev=1690026&r1=1690025&r2=1690026&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java Thu Jul  9 08:39:00 2015
@@ -30,8 +30,8 @@ import org.apache.lucene.index.TermState
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.BooleanClause.Occur;
-import org.apache.lucene.util.BitDocIdSet;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.DocIdSetBuilder;
 
 /**
  * This class also provides the functionality behind
@@ -60,17 +60,17 @@ final class MultiTermQueryConstantScoreW
     }
   }
 
-  private static class WeightOrBitSet {
+  private static class WeightOrDocIdSet {
     final Weight weight;
-    final BitDocIdSet bitset;
+    final DocIdSet set;
 
-    WeightOrBitSet(Weight weight) {
+    WeightOrDocIdSet(Weight weight) {
       this.weight = Objects.requireNonNull(weight);
-      this.bitset = null;
+      this.set = null;
     }
 
-    WeightOrBitSet(BitDocIdSet bitset) {
-      this.bitset = bitset;
+    WeightOrDocIdSet(DocIdSet bitset) {
+      this.set = bitset;
       this.weight = null;
     }
   }
@@ -135,11 +135,11 @@ final class MultiTermQueryConstantScoreW
        * On the given leaf context, try to either rewrite to a disjunction if
        * there are few terms, or build a bitset containing matching docs.
        */
-      private WeightOrBitSet rewrite(LeafReaderContext context) throws IOException {
+      private WeightOrDocIdSet rewrite(LeafReaderContext context) throws IOException {
         final Terms terms = context.reader().terms(query.field);
         if (terms == null) {
           // field does not exist
-          return new WeightOrBitSet((BitDocIdSet) null);
+          return new WeightOrDocIdSet((DocIdSet) null);
         }
 
         final TermsEnum termsEnum = query.getTermsEnum(terms);
@@ -158,30 +158,30 @@ final class MultiTermQueryConstantScoreW
           }
           Query q = new ConstantScoreQuery(bq.build());
           q.setBoost(score());
-          return new WeightOrBitSet(searcher.rewrite(q).createWeight(searcher, needsScores));
+          return new WeightOrDocIdSet(searcher.rewrite(q).createWeight(searcher, needsScores));
         }
 
         // Too many terms: go back to the terms we already collected and start building the bit set
-        BitDocIdSet.Builder builder = new BitDocIdSet.Builder(context.reader().maxDoc());
+        DocIdSetBuilder builder = new DocIdSetBuilder(context.reader().maxDoc());
         if (collectedTerms.isEmpty() == false) {
           TermsEnum termsEnum2 = terms.iterator();
           for (TermAndState t : collectedTerms) {
             termsEnum2.seekExact(t.term, t.state);
             docs = termsEnum2.postings(docs, PostingsEnum.NONE);
-            builder.or(docs);
+            builder.add(docs);
           }
         }
 
         // Then keep filling the bit set with remaining terms
         do {
           docs = termsEnum.postings(docs, PostingsEnum.NONE);
-          builder.or(docs);
+          builder.add(docs);
         } while (termsEnum.next() != null);
 
-        return new WeightOrBitSet(builder.build());
+        return new WeightOrDocIdSet(builder.build());
       }
 
-      private Scorer scorer(BitDocIdSet set) {
+      private Scorer scorer(DocIdSet set) throws IOException {
         if (set == null) {
           return null;
         }
@@ -194,11 +194,11 @@ final class MultiTermQueryConstantScoreW
 
       @Override
       public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
-        final WeightOrBitSet weightOrBitSet = rewrite(context);
+        final WeightOrDocIdSet weightOrBitSet = rewrite(context);
         if (weightOrBitSet.weight != null) {
           return weightOrBitSet.weight.bulkScorer(context);
         } else {
-          final Scorer scorer = scorer(weightOrBitSet.bitset);
+          final Scorer scorer = scorer(weightOrBitSet.set);
           if (scorer == null) {
             return null;
           }
@@ -208,11 +208,11 @@ final class MultiTermQueryConstantScoreW
 
       @Override
       public Scorer scorer(LeafReaderContext context) throws IOException {
-        final WeightOrBitSet weightOrBitSet = rewrite(context);
+        final WeightOrDocIdSet weightOrBitSet = rewrite(context);
         if (weightOrBitSet.weight != null) {
           return weightOrBitSet.weight.scorer(context);
         } else {
-          return scorer(weightOrBitSet.bitset);
+          return scorer(weightOrBitSet.set);
         }
       }
     };

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BitDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BitDocIdSet.java?rev=1690026&r1=1690025&r2=1690026&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BitDocIdSet.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BitDocIdSet.java Thu Jul  9 08:39:00 2015
@@ -17,8 +17,6 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
-import java.io.IOException;
-
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
 
@@ -70,105 +68,4 @@ public class BitDocIdSet extends DocIdSe
     return getClass().getSimpleName() + "(set=" + set + ",cost=" + cost + ")";
   }
 
-  /**
-   * A builder of {@link DocIdSet}s that supports random access.
-   * @lucene.internal
-   */
-  public static final class Builder {
-
-    private final int maxDoc;
-    private final int threshold;
-    private SparseFixedBitSet sparseSet;
-    private FixedBitSet denseSet;
-
-    // we cache an upper bound of the cost of this builder so that we don't have
-    // to re-compute approximateCardinality on the sparse set every time 
-    private long costUpperBound;
-
-    /** Create a new instance that can hold <code>maxDoc</code> documents and is optionally <code>full</code>. */
-    public Builder(int maxDoc, boolean full) {
-      this.maxDoc = maxDoc;
-      threshold = maxDoc >>> 10;
-      if (full) {
-        denseSet = new FixedBitSet(maxDoc);
-        denseSet.set(0, maxDoc);
-      }
-    }
-
-    /** Create a new empty instance. */
-    public Builder(int maxDoc) {
-      this(maxDoc, false);
-    }
-
-    // pkg-private for testing
-    boolean dense() {
-      return denseSet != null;
-    }
-
-    /**
-     * Is this builder definitely empty?  If so, {@link #build()} will return null.  This is usually the same as
-     * simply being empty but if this builder was constructed with the {@code full} option or if an iterator was passed
-     * that iterated over no documents, then we're not sure.
-     */
-    public boolean isDefinitelyEmpty() {
-      return sparseSet == null && denseSet == null;
-    }
-
-    /**
-     * Add the content of the provided {@link DocIdSetIterator} to this builder.
-     */
-    public void or(DocIdSetIterator it) throws IOException {
-      if (denseSet != null) {
-        // already upgraded
-        denseSet.or(it);
-        return;
-      }
-
-      final long itCost = it.cost();
-      costUpperBound += itCost;
-      if (costUpperBound >= threshold) {
-        costUpperBound = (sparseSet == null ? 0 : sparseSet.approximateCardinality()) + itCost;
-
-        if (costUpperBound >= threshold) {
-          // upgrade
-          denseSet = new FixedBitSet(maxDoc);
-          denseSet.or(it);
-          if (sparseSet != null) {
-            denseSet.or(new BitSetIterator(sparseSet, 0L));
-          }
-          return;
-        }
-      }
-
-      // we are still sparse
-      if (sparseSet == null) {
-        sparseSet = new SparseFixedBitSet(maxDoc);
-      }
-      sparseSet.or(it);
-    }
-
-    /**
-     * Build a {@link DocIdSet} that contains all doc ids that have been added.
-     * This method may return <tt>null</tt> if no documents were addded to this
-     * builder.
-     * NOTE: this is a destructive operation, the builder should not be used
-     * anymore after this method has been called.
-     */
-    public BitDocIdSet build() {
-      final BitDocIdSet result;
-      if (denseSet != null) {
-        result = new BitDocIdSet(denseSet);
-      } else if (sparseSet != null) {
-        result = new BitDocIdSet(sparseSet);
-      } else {
-        result = null;
-      }
-      denseSet = null;
-      sparseSet = null;
-      costUpperBound = 0;
-      return result;
-    }
-
-  }
-
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BitSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BitSet.java?rev=1690026&r1=1690025&r2=1690026&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BitSet.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BitSet.java Thu Jul  9 08:39:00 2015
@@ -27,6 +27,21 @@ import org.apache.lucene.search.DocIdSet
  */
 public abstract class BitSet implements MutableBits, Accountable {
 
+  /** Build a {@link BitSet} from the content of the provided {@link DocIdSetIterator}.
+   *  NOTE: this will consume the {@link BitSet}. */
+  public static BitSet of(DocIdSetIterator it, int maxDoc) throws IOException {
+    final long cost = it.cost();
+    final int threshold = maxDoc >>> 7;
+    BitSet set;
+    if (cost < threshold) {
+      set = new SparseFixedBitSet(maxDoc);
+    } else {
+      set = new FixedBitSet(maxDoc);
+    }
+    set.or(it);
+    return set;
+  }
+
   /** Set the bit at <code>i</code>. */
   public abstract void set(int i);
 

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java?rev=1690026&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java Thu Jul  9 08:39:00 2015
@@ -0,0 +1,189 @@
+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.io.IOException;
+
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.DocIdSetIterator;
+
+/**
+ * A builder of {@link DocIdSet}s.
+ * @lucene.internal
+ */
+public final class DocIdSetBuilder {
+
+  private final int maxDoc;
+  private final int threshold;
+
+  private int[] buffer;
+  private int bufferSize;
+
+  private BitSet bitSet;
+
+  /**
+   * Create a builder that can contain doc IDs between {@code 0} and {@code maxDoc}.
+   */
+  public DocIdSetBuilder(int maxDoc) {
+    this.maxDoc = maxDoc;
+    // For ridiculously small sets, we'll just use a sorted int[]
+    // maxDoc >>> 7 is a good value if you want to save memory, lower values
+    // such as maxDoc >>> 11 should provide faster building but at the expense
+    // of using a full bitset even for quite sparse data
+    this.threshold = maxDoc >>> 7;
+
+    this.buffer = new int[0];
+    this.bufferSize = 0;
+    this.bitSet = null;
+  }
+
+  private void upgradeToBitSet() {
+    assert bitSet == null;
+    bitSet = new FixedBitSet(maxDoc);
+    for (int i = 0; i < bufferSize; ++i) {
+      bitSet.set(buffer[i]);
+    }
+    this.buffer = null;
+    this.bufferSize = 0;
+  }
+
+  /**
+   * Add the content of the provided {@link DocIdSetIterator} to this builder.
+   * NOTE: if you need to build a {@link DocIdSet} out of a single
+   * {@link DocIdSetIterator}, you should rather use {@link RoaringDocIdSet.Builder}.
+   */
+  public void add(DocIdSetIterator iter) throws IOException {
+    grow((int) Math.min(Integer.MAX_VALUE, iter.cost()));
+
+    if (bitSet != null) {
+      bitSet.or(iter);
+    } else {
+      while (true) {  
+        final int end = Math.min(threshold, buffer.length);
+        for (int i = bufferSize; i < end; ++i) {
+          final int doc = iter.nextDoc();
+          if (doc == DocIdSetIterator.NO_MORE_DOCS) {
+            bufferSize = i;
+            return;
+          }
+          buffer[bufferSize++] = doc;
+        }
+        bufferSize = end;
+
+        if (bufferSize + 1 >= threshold) {
+          break;
+        }
+
+        buffer = ArrayUtil.grow(buffer, bufferSize + 1);
+      }
+
+      upgradeToBitSet();
+      for (int doc = iter.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = iter.nextDoc()) {
+        bitSet.set(doc);
+      }
+    }
+  }
+
+  /**
+   * Reserve space so that this builder can hold {@code numDocs} MORE documents.
+   */
+  public void grow(int numDocs) {
+    if (bitSet == null) {
+      final long newLength = bufferSize + numDocs;
+      if (newLength <= threshold) {
+        buffer = ArrayUtil.grow(buffer, (int) newLength);
+      } else {
+        upgradeToBitSet();
+      }
+    }
+  }
+
+  /**
+   * Add a document to this builder.
+   * NOTE: doc IDs do not need to be provided in order.
+   * NOTE: if you plan on adding several docs at once, look into using
+   * {@link #grow(int)} to reserve space.
+   */
+  public void add(int doc) {
+    if (bitSet != null) {
+      bitSet.set(doc);
+    } else {
+      if (bufferSize + 1 >= threshold) {
+        upgradeToBitSet();
+        bitSet.set(doc);
+        return;
+      }
+      if (bufferSize + 1 > buffer.length) {
+        buffer = ArrayUtil.grow(buffer, bufferSize + 1);
+      }
+      buffer[bufferSize++] = doc;
+    }
+  }
+
+  private static int dedup(int[] arr, int length) {
+    if (length == 0) {
+      return 0;
+    }
+    int l = 1;
+    int previous = arr[0];
+    for (int i = 1; i < length; ++i) {
+      final int value = arr[i];
+      assert value >= previous;
+      if (value != previous) {
+        arr[l++] = value;
+        previous = value;
+      }
+    }
+    return l;
+  }
+
+  /**
+   * Build a {@link DocIdSet} from the accumulated doc IDs.
+   */
+  public DocIdSet build() {
+    return build(-1);
+  }
+
+  /**
+   * Expert: build a {@link DocIdSet} with a hint on the cost that the resulting
+   * {@link DocIdSet} would have.
+   */
+  public DocIdSet build(long costHint) {
+    try {
+      if (bitSet != null) {
+        if (costHint == -1) {
+          return new BitDocIdSet(bitSet);
+        } else {
+          return new BitDocIdSet(bitSet, costHint);
+        }
+      } else {
+        LSBRadixSorter sorter = new LSBRadixSorter();
+        sorter.sort(buffer, 0, bufferSize);
+        final int l = dedup(buffer, bufferSize);
+        buffer = ArrayUtil.grow(buffer, l + 1);
+        buffer[l] = DocIdSetIterator.NO_MORE_DOCS;
+        return new IntArrayDocIdSet(buffer, l);
+      }
+    } finally {
+      this.buffer = null;
+      this.bufferSize = 0;
+      this.bitSet = null;
+    }
+  }
+
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/IntArrayDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/IntArrayDocIdSet.java?rev=1690026&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/IntArrayDocIdSet.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/IntArrayDocIdSet.java Thu Jul  9 08:39:00 2015
@@ -0,0 +1,89 @@
+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.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.DocIdSetIterator;
+
+final class IntArrayDocIdSet extends DocIdSet {
+
+  private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(IntArrayDocIdSet.class);
+
+  private final int[] docs;
+  private final int length;
+
+  IntArrayDocIdSet(int[] docs, int length) {
+    if (docs[length] != DocIdSetIterator.NO_MORE_DOCS) {
+      throw new IllegalArgumentException();
+    }
+    this.docs = docs;
+    this.length = length;
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return BASE_RAM_BYTES_USED + RamUsageEstimator.sizeOf(docs);
+  }
+
+  @Override
+  public DocIdSetIterator iterator() throws IOException {
+    return new IntArrayDocIdSetIterator(docs, length);
+  }
+
+  static class IntArrayDocIdSetIterator extends DocIdSetIterator {
+
+    private final int[] docs;
+    private final int length;
+    private int i = -1;
+    private int doc = -1;
+
+    IntArrayDocIdSetIterator(int[] docs, int length) {
+      this.docs = docs;
+      this.length = length;
+    }
+
+    @Override
+    public int docID() {
+      return doc;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      return doc = docs[++i];
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      i = Arrays.binarySearch(docs, i + 1, length, target);
+      if (i < 0) {
+        i = -1 - i;
+      }
+      return doc = docs[i];
+    }
+
+    @Override
+    public long cost() {
+      return length;
+    }
+
+  }
+
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/LSBRadixSorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/LSBRadixSorter.java?rev=1690026&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/LSBRadixSorter.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/LSBRadixSorter.java Thu Jul  9 08:39:00 2015
@@ -0,0 +1,113 @@
+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;
+
+/**
+ * A LSB Radix sorter for unsigned int values.
+ */
+final class LSBRadixSorter {
+
+  private static final int INSERTION_SORT_THRESHOLD = 30;
+  private static final int HISTOGRAM_SIZE = 256;
+
+  private final int[] histogram = new int[HISTOGRAM_SIZE];
+  private int[] buffer = new int[0];
+
+  private static void buildHistogram(int[] array, int off, int len, int[] histogram, int shift) {
+    for (int i = 0; i < len; ++i) {
+      final int b = (array[off + i] >>> shift) & 0xFF;
+      histogram[b] += 1;
+    }
+  }
+
+  private static void sumHistogram(int[] histogram) {
+    int accum = 0;
+    for (int i = 0; i < HISTOGRAM_SIZE; ++i) {
+      final int count = histogram[i];
+      histogram[i] = accum;
+      accum += count;
+    }
+  }
+
+  private static void reorder(int[] array, int off, int len, int[] histogram, int shift, int[] dest, int destOff) {
+    for (int i = 0; i < len; ++i) {
+      final int v = array[off + i];
+      final int b = (v >>> shift) & 0xFF;
+      dest[destOff + histogram[b]++] = v;
+    }
+  }
+
+  private static boolean sort(int[] array, int off, int len, int[] histogram, int shift, int[] dest, int destOff) {
+    Arrays.fill(histogram, 0);
+    buildHistogram(array, off, len, histogram, shift);
+    if (histogram[0] == len) {
+      return false;
+    }
+    sumHistogram(histogram);
+    reorder(array, off, len, histogram, shift, dest, destOff);
+    return true;
+  }
+
+  private static void insertionSort(int[] array, int off, int len) {
+    for (int i = off + 1, end = off + len; i < end; ++i) {
+      for (int j = i; j > off; --j) {
+        if (array[j - 1] > array[j]) {
+          int tmp = array[j - 1];
+          array[j - 1] = array[j];
+          array[j] = tmp;
+        } else {
+          break;
+        }
+      }
+    }
+  }
+
+  public void sort(final int[] array, int off, int len) {
+    if (len < INSERTION_SORT_THRESHOLD) {
+      insertionSort(array, off, len);
+      return;
+    }
+
+    buffer = ArrayUtil.grow(buffer, len);
+
+    int[] arr = array;
+    int arrOff = off;
+
+    int[] buf = buffer;
+    int bufOff = 0;
+    
+    for (int shift = 0; shift <= 24; shift += 8) {
+      if (sort(arr, arrOff, len, histogram, shift, buf, bufOff)) {
+        // swap arrays
+        int[] tmp = arr;
+        int tmpOff = arrOff;
+        arr = buf;
+        arrOff = bufOff;
+        buf = tmp;
+        bufOff = tmpOff;
+      }
+    }
+
+    if (array == buf) {
+      System.arraycopy(arr, arrOff, array, off, len);
+    }
+  }
+
+}

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java?rev=1690026&r1=1690025&r2=1690026&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java Thu Jul  9 08:39:00 2015
@@ -22,10 +22,11 @@ import java.io.IOException;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
 
+
 public class TestDocIdSetBuilder extends LuceneTestCase {
 
   public void testEmpty() throws IOException {
-    assertEquals(null, new BitDocIdSet.Builder(1 + random().nextInt(1000)).build());
+    assertEquals(null, new DocIdSetBuilder(1 + random().nextInt(1000)).build());
   }
 
   private void assertEquals(DocIdSet d1, DocIdSet d2) throws IOException {
@@ -45,19 +46,9 @@ public class TestDocIdSetBuilder extends
     }
   }
 
-  public void testFull() throws IOException {
-    final int maxDoc = 1 + random().nextInt(1000);
-    BitDocIdSet.Builder builder = new BitDocIdSet.Builder(maxDoc, true);
-    DocIdSet set = builder.build();
-    DocIdSetIterator it = set.iterator();
-    for (int i = 0; i < maxDoc; ++i) {
-      assertEquals(i, it.nextDoc());
-    }
-  }
-
   public void testSparse() throws IOException {
     final int maxDoc = 1000000 + random().nextInt(1000000);
-    BitDocIdSet.Builder builder = new BitDocIdSet.Builder(maxDoc);
+    DocIdSetBuilder builder = new DocIdSetBuilder(maxDoc);
     final int numIterators = 1 + random().nextInt(10);
     final FixedBitSet ref = new FixedBitSet(maxDoc);
     for (int i = 0; i < numIterators; ++i) {
@@ -67,35 +58,101 @@ public class TestDocIdSetBuilder extends
         b.add(doc);
         ref.set(doc);
       }
-      builder.or(b.build().iterator());
+      builder.add(b.build().iterator());
     }
     DocIdSet result = builder.build();
-    assertTrue(result instanceof BitDocIdSet);
+    assertTrue(result instanceof IntArrayDocIdSet);
     assertEquals(new BitDocIdSet(ref), result);
   }
 
   public void testDense() throws IOException {
     final int maxDoc = 1000000 + random().nextInt(1000000);
-    BitDocIdSet.Builder builder = new BitDocIdSet.Builder(maxDoc);
+    DocIdSetBuilder builder = new DocIdSetBuilder(maxDoc);
     final int numIterators = 1 + random().nextInt(10);
     final FixedBitSet ref = new FixedBitSet(maxDoc);
-    if (random().nextBoolean()) {
-      // try upgrades
-      final int doc = random().nextInt(maxDoc);
-      ref.set(doc);
-      builder.or(new RoaringDocIdSet.Builder(maxDoc).add(doc).build().iterator());
-    }
     for (int i = 0; i < numIterators; ++i) {
       RoaringDocIdSet.Builder b = new RoaringDocIdSet.Builder(maxDoc);
-      for (int doc = random().nextInt(1000); doc < maxDoc; doc += 1 + random().nextInt(1000)) {
+      for (int doc = random().nextInt(1000); doc < maxDoc; doc += 1 + random().nextInt(100)) {
         b.add(doc);
         ref.set(doc);
       }
-      builder.or(b.build().iterator());
+      builder.add(b.build().iterator());
     }
     DocIdSet result = builder.build();
     assertTrue(result instanceof BitDocIdSet);
     assertEquals(new BitDocIdSet(ref), result);
   }
 
+  public void testRandom() throws IOException {
+    final int maxDoc = TestUtil.nextInt(random(), 1, 10000000);
+    for (int i = 1 ; i < maxDoc / 2; i <<=1) {
+      final int numDocs = TestUtil.nextInt(random(), 1, i);
+      final FixedBitSet docs = new FixedBitSet(maxDoc);
+      int c = 0;
+      while (c < numDocs) {
+        final int d = random().nextInt(maxDoc);
+        if (docs.get(d) == false) {
+          docs.set(d);
+          c += 1;
+        }
+      }
+
+      final int[] array = new int[numDocs + random().nextInt(100)];
+      DocIdSetIterator it = new BitSetIterator(docs, 0L);
+      int j = 0;
+      for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
+        array[j++] = doc;
+      }
+      assertEquals(numDocs, j);
+
+      // add some duplicates
+      while (j < array.length) {
+        array[j++] = array[random().nextInt(numDocs)];
+      }
+
+      // shuffle
+      for (j = array.length - 1; j >= 1; --j) {
+        final int k = random().nextInt(j);
+        int tmp = array[j];
+        array[j] = array[k];
+        array[k] = tmp;
+      }
+
+      // add docs out of order
+      DocIdSetBuilder builder = new DocIdSetBuilder(maxDoc);
+      for (j = 0; j < array.length; ) {
+        final int l = TestUtil.nextInt(random(), 1, array.length - j);
+        if (rarely()) {
+          builder.grow(l);
+        }
+        for (int k = 0; k < l; ++k) {
+          builder.add(array[j++]);
+        }
+      }
+
+      final DocIdSet expected = new BitDocIdSet(docs);
+      final DocIdSet actual = builder.build();
+      assertEquals(expected, actual);
+    }
+  }
+
+  public void testMisleadingDISICost() throws IOException {
+    final int maxDoc = TestUtil.nextInt(random(), 1000, 10000);
+    DocIdSetBuilder builder = new DocIdSetBuilder(maxDoc);
+    FixedBitSet expected = new FixedBitSet(maxDoc);
+
+    for (int i = 0; i < 10; ++i) {
+      final FixedBitSet docs = new FixedBitSet(maxDoc);
+      final int numDocs = random().nextInt(maxDoc / 1000);
+      for (int j = 0; j < numDocs; ++j) {
+        docs.set(random().nextInt(maxDoc));
+      }
+      expected.or(docs);
+      // We provide a cost of 0 here to make sure the builder can deal with wrong costs
+      builder.add(new BitSetIterator(docs, 0L));
+    }
+
+    assertEquals(new BitDocIdSet(expected), builder.build());
+  }
+
 }

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestIntArrayDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestIntArrayDocIdSet.java?rev=1690026&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestIntArrayDocIdSet.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestIntArrayDocIdSet.java Thu Jul  9 08:39:00 2015
@@ -0,0 +1,40 @@
+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.io.IOException;
+import java.util.BitSet;
+
+import org.apache.lucene.search.DocIdSetIterator;
+
+public class TestIntArrayDocIdSet extends BaseDocIdSetTestCase<IntArrayDocIdSet> {
+
+  @Override
+  public IntArrayDocIdSet copyOf(BitSet bs, int length) throws IOException {
+    int[] docs = new int[0];
+    int l = 0;
+    for (int i = bs.nextSetBit(0); i != -1; i = bs.nextSetBit(i + 1)) {
+      docs = ArrayUtil.grow(docs, length + 1);
+      docs[l++] = i;
+    }
+    docs = ArrayUtil.grow(docs, length + 1);
+    docs[l] = DocIdSetIterator.NO_MORE_DOCS;
+    return new IntArrayDocIdSet(docs, l);
+  }
+  
+}

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestLSBRadixSorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestLSBRadixSorter.java?rev=1690026&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestLSBRadixSorter.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestLSBRadixSorter.java Thu Jul  9 08:39:00 2015
@@ -0,0 +1,81 @@
+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 class TestLSBRadixSorter extends LuceneTestCase {
+
+  public void test(LSBRadixSorter sorter, int maxLen) {
+    for (int iter = 0; iter < 10; ++iter) {
+      int off = random().nextInt(10);
+      final int len = TestUtil.nextInt(random(), 0, maxLen);
+      int[] arr = new int[off + len + random().nextInt(10)];
+      final int numBits = random().nextInt(31);
+      final int maxValue = (1 << numBits) - 1;
+      for (int i = 0; i < arr.length; ++i) {
+        arr[i] = TestUtil.nextInt(random(), 0, maxValue);
+      }
+      test(sorter, arr, off, len);
+    }
+  }
+
+  public void test(LSBRadixSorter sorter, int[] arr, int off, int len) {
+    final int[] expected = Arrays.copyOfRange(arr, off, off + len);
+    Arrays.sort(expected);
+
+    sorter.sort(arr, off, len);
+    final int[] actual = Arrays.copyOfRange(arr, off, off + len);
+    assertArrayEquals(expected, actual);
+  }
+
+  public void testEmpty() {
+    test(new LSBRadixSorter(), 0);
+  }
+
+  public void testOne() {
+    test(new LSBRadixSorter(), 1);
+  }
+
+  public void testTwo() {
+    test(new LSBRadixSorter(), 2);
+  }
+
+  public void testSimple() {
+    test(new LSBRadixSorter(), 100);
+  }
+
+  public void testRandom() {
+    test(new LSBRadixSorter(), 10000);
+  }
+
+  public void testSorted() {
+    LSBRadixSorter sorter = new LSBRadixSorter();
+    for (int iter = 0; iter < 10; ++iter) {
+      int[] arr = new int[10000];
+      int a = 0;
+      for (int i = 0; i < arr.length; ++i) {
+        a += random().nextInt(10);
+        arr[i] = a;
+      }
+      final int off = random().nextInt(arr.length);
+      final int len = TestUtil.nextInt(random(), 0, arr.length - off);
+      test(sorter, arr, off, len);
+    }
+  }
+}

Modified: lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/QueryBitSetProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/QueryBitSetProducer.java?rev=1690026&r1=1690025&r2=1690026&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/QueryBitSetProducer.java (original)
+++ lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/QueryBitSetProducer.java Thu Jul  9 08:39:00 2015
@@ -70,14 +70,10 @@ public class QueryBitSetProducer impleme
       final Weight weight = searcher.createNormalizedWeight(query, false);
       final DocIdSetIterator it = weight.scorer(context);
 
-      BitDocIdSet.Builder builder = new BitDocIdSet.Builder(context.reader().maxDoc());
-      if (it != null) {
-        builder.or(it);
-      }
-      docIdSet = builder.build();
-      if (docIdSet == null) {
-        // We use EMPTY as a sentinel for the empty set, which is cacheable
+      if (it == null) {
         docIdSet = DocIdSet.EMPTY;
+      } else {
+        docIdSet = new BitDocIdSet(BitSet.of(it, context.reader().maxDoc()));
       }
       cache.put(key, docIdSet);
     }

Modified: lucene/dev/trunk/lucene/misc/src/test/org/apache/lucene/index/TestBlockJoinSorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/misc/src/test/org/apache/lucene/index/TestBlockJoinSorter.java?rev=1690026&r1=1690025&r2=1690026&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/misc/src/test/org/apache/lucene/index/TestBlockJoinSorter.java (original)
+++ lucene/dev/trunk/lucene/misc/src/test/org/apache/lucene/index/TestBlockJoinSorter.java Thu Jul  9 08:39:00 2015
@@ -48,6 +48,7 @@ import org.apache.lucene.search.SortFiel
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BitDocIdSet;
+import org.apache.lucene.util.BitSet;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.LuceneTestCase;
@@ -74,9 +75,7 @@ public class TestBlockJoinSorter extends
         final DocIdSet uncached = filter.getDocIdSet(context, null);
         final DocIdSetIterator it = uncached == null ? null : uncached.iterator();
         if (it != null) {
-          BitDocIdSet.Builder builder = new BitDocIdSet.Builder(context.reader().maxDoc());
-          builder.or(it);
-          docIdSet = builder.build();
+          docIdSet = new BitDocIdSet(BitSet.of(it, context.reader().maxDoc()));
         }
         if (docIdSet == null) {
           docIdSet = new BitDocIdSet(new SparseFixedBitSet(context.reader().maxDoc()));

Modified: lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java?rev=1690026&r1=1690025&r2=1690026&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java (original)
+++ lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java Thu Jul  9 08:39:00 2015
@@ -44,6 +44,7 @@ import org.apache.lucene.search.BulkScor
 import org.apache.lucene.search.ConstantScoreQuery;
 import org.apache.lucene.search.ConstantScoreScorer;
 import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
@@ -52,9 +53,8 @@ import org.apache.lucene.search.TermQuer
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.BitDocIdSet;
-import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.DocIdSetBuilder;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.ToStringUtils;
 
@@ -221,17 +221,17 @@ public class TermsQuery extends Query im
     }
   }
 
-  private static class WeightOrBitSet {
+  private static class WeightOrDocIdSet {
     final Weight weight;
-    final BitDocIdSet bitset;
+    final DocIdSet set;
 
-    WeightOrBitSet(Weight weight) {
+    WeightOrDocIdSet(Weight weight) {
       this.weight = Objects.requireNonNull(weight);
-      this.bitset = null;
+      this.set = null;
     }
 
-    WeightOrBitSet(BitDocIdSet bitset) {
-      this.bitset = bitset;
+    WeightOrDocIdSet(DocIdSet bitset) {
+      this.set = bitset;
       this.weight = null;
     }
   }
@@ -252,7 +252,7 @@ public class TermsQuery extends Query im
        * On the given leaf context, try to either rewrite to a disjunction if
        * there are few matching terms, or build a bitset containing matching docs.
        */
-      private WeightOrBitSet rewrite(LeafReaderContext context) throws IOException {
+      private WeightOrDocIdSet rewrite(LeafReaderContext context) throws IOException {
         final LeafReader reader = context.reader();
 
         // We will first try to collect up to 'threshold' terms into 'matchingTerms'
@@ -260,7 +260,7 @@ public class TermsQuery extends Query im
         final int threshold = Math.min(BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD, BooleanQuery.getMaxClauseCount());
         assert termData.size() > threshold : "Query should have been rewritten";
         List<TermAndState> matchingTerms = new ArrayList<>(threshold);
-        BitDocIdSet.Builder builder = null;
+        DocIdSetBuilder builder = null;
 
         final Fields fields = reader.fields();
         String lastField = null;
@@ -283,18 +283,18 @@ public class TermsQuery extends Query im
           if (termsEnum != null && termsEnum.seekExact(term)) {
             if (matchingTerms == null) {
               docs = termsEnum.postings(docs, PostingsEnum.NONE);
-              builder.or(docs);
+              builder.add(docs);
             } else if (matchingTerms.size() < threshold) {
               matchingTerms.add(new TermAndState(field, termsEnum));
             } else {
               assert matchingTerms.size() == threshold;
-              builder = new BitDocIdSet.Builder(reader.maxDoc());
+              builder = new DocIdSetBuilder(reader.maxDoc());
               docs = termsEnum.postings(docs, PostingsEnum.NONE);
-              builder.or(docs);
+              builder.add(docs);
               for (TermAndState t : matchingTerms) {
                 t.termsEnum.seekExact(t.term, t.state);
                 docs = t.termsEnum.postings(docs, PostingsEnum.NONE);
-                builder.or(docs);
+                builder.add(docs);
               }
               matchingTerms = null;
             }
@@ -310,14 +310,14 @@ public class TermsQuery extends Query im
           }
           Query q = new ConstantScoreQuery(bq.build());
           q.setBoost(score());
-          return new WeightOrBitSet(searcher.rewrite(q).createWeight(searcher, needsScores));
+          return new WeightOrDocIdSet(searcher.rewrite(q).createWeight(searcher, needsScores));
         } else {
           assert builder != null;
-          return new WeightOrBitSet(builder.build());
+          return new WeightOrDocIdSet(builder.build());
         }
       }
 
-      private Scorer scorer(BitDocIdSet set) {
+      private Scorer scorer(DocIdSet set) throws IOException {
         if (set == null) {
           return null;
         }
@@ -330,11 +330,11 @@ public class TermsQuery extends Query im
 
       @Override
       public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
-        final WeightOrBitSet weightOrBitSet = rewrite(context);
+        final WeightOrDocIdSet weightOrBitSet = rewrite(context);
         if (weightOrBitSet.weight != null) {
           return weightOrBitSet.weight.bulkScorer(context);
         } else {
-          final Scorer scorer = scorer(weightOrBitSet.bitset);
+          final Scorer scorer = scorer(weightOrBitSet.set);
           if (scorer == null) {
             return null;
           }
@@ -344,11 +344,11 @@ public class TermsQuery extends Query im
 
       @Override
       public Scorer scorer(LeafReaderContext context) throws IOException {
-        final WeightOrBitSet weightOrBitSet = rewrite(context);
+        final WeightOrDocIdSet weightOrBitSet = rewrite(context);
         if (weightOrBitSet.weight != null) {
           return weightOrBitSet.weight.scorer(context);
         } else {
-          return scorer(weightOrBitSet.bitset);
+          return scorer(weightOrBitSet.set);
         }
       }
     };

Modified: lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeReader.java?rev=1690026&r1=1690025&r2=1690026&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeReader.java (original)
+++ lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeReader.java Thu Jul  9 08:39:00 2015
@@ -24,9 +24,7 @@ import org.apache.lucene.search.DocIdSet
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Accountable;
-import org.apache.lucene.util.BitDocIdSet;
-import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.DocIdSetBuilder;
 import org.apache.lucene.util.RamUsageEstimator;
 
 /** Handles intersection of a shape with a BKD tree previously written with {@link BKDTreeWriter}.
@@ -71,7 +69,7 @@ final class BKDTreeReader implements Acc
     final IndexInput in;
     byte[] scratch = new byte[16];
     final ByteArrayDataInput scratchReader = new ByteArrayDataInput(scratch);
-    final FixedBitSet bits;
+    final DocIdSetBuilder docs;
     final int latMinEnc;
     final int latMaxEnc;
     final int lonMinEnc;
@@ -85,7 +83,7 @@ final class BKDTreeReader implements Acc
                       LatLonFilter latLonFilter,
                       SortedNumericDocValues sndv) {
       this.in = in;
-      this.bits = new FixedBitSet(maxDoc);
+      this.docs = new DocIdSetBuilder(maxDoc);
       this.latMinEnc = latMinEnc;
       this.latMaxEnc = latMaxEnc;
       this.lonMinEnc = lonMinEnc;
@@ -135,7 +133,7 @@ final class BKDTreeReader implements Acc
                              BKDTreeWriter.encodeLon(Math.nextAfter(180.0, Double.POSITIVE_INFINITY)));
 
     // NOTE: hitCount is an over-estimate in the multi-valued case:
-    return new BitDocIdSet(state.bits, hitCount);
+    return state.docs.build(hitCount);
   }
 
   /** Fast path: this is called when the query rect fully encompasses all cells under this node. */
@@ -167,9 +165,10 @@ final class BKDTreeReader implements Acc
       //System.out.println("    seek to leafFP=" + fp);
       // How many points are stored in this leaf cell:
       int count = state.in.readVInt();
+      state.docs.grow(count);
       for(int i=0;i<count;i++) {
         int docID = state.in.readInt();
-        state.bits.set(docID);
+        state.docs.add(docID);
       }
 
       //bits.or(allLeafDISI);
@@ -262,6 +261,7 @@ final class BKDTreeReader implements Acc
       // How many points are stored in this leaf cell:
       int count = state.in.readVInt();
 
+      state.docs.grow(count);
       for(int i=0;i<count;i++) {
         int docID = state.in.readInt();
         state.sndv.setDocument(docID);
@@ -279,7 +279,7 @@ final class BKDTreeReader implements Acc
               lonEnc < state.lonMaxEnc &&
               (state.latLonFilter == null ||
                state.latLonFilter.accept(BKDTreeWriter.decodeLat(latEnc), BKDTreeWriter.decodeLon(lonEnc)))) {
-            state.bits.set(docID);
+            state.docs.add(docID);
             hitCount++;
 
             // Stop processing values for this doc:

Modified: lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java?rev=1690026&r1=1690025&r2=1690026&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java (original)
+++ lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java Thu Jul  9 08:39:00 2015
@@ -22,6 +22,7 @@ import java.util.Map;
 
 import com.spatial4j.core.shape.Shape;
 import com.spatial4j.core.shape.SpatialRelation;
+
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
@@ -37,6 +38,7 @@ import org.apache.lucene.search.Weight;
 import org.apache.lucene.spatial.prefix.AbstractVisitingPrefixTreeFilter;
 import org.apache.lucene.spatial.prefix.tree.Cell;
 import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
+import org.apache.lucene.spatial.util.BitDocIdSetBuilder;
 import org.apache.lucene.util.BitDocIdSet;
 import org.apache.lucene.util.Bits;
 
@@ -151,8 +153,8 @@ public class IntersectsRPTVerifyQuery ex
     // TODO consider if IntersectsPrefixTreeFilter should simply do this and provide both sets
 
     class IntersectsDifferentiatingVisitor extends VisitorTemplate {
-      BitDocIdSet.Builder approxBuilder = new BitDocIdSet.Builder(maxDoc);
-      BitDocIdSet.Builder exactBuilder = new BitDocIdSet.Builder(maxDoc);
+      BitDocIdSetBuilder approxBuilder = new BitDocIdSetBuilder(maxDoc);
+      BitDocIdSetBuilder exactBuilder = new BitDocIdSetBuilder(maxDoc);
       BitDocIdSet exactDocIdSet;
       BitDocIdSet approxDocIdSet;
 

Modified: lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeFilter.java?rev=1690026&r1=1690025&r2=1690026&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeFilter.java (original)
+++ lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeFilter.java Thu Jul  9 08:39:00 2015
@@ -27,10 +27,9 @@ import org.apache.lucene.index.LeafReade
 import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.Filter;
 import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
-import org.apache.lucene.util.BitDocIdSet;
+import org.apache.lucene.spatial.util.BitDocIdSetBuilder;
 import org.apache.lucene.util.BitSet;
 import org.apache.lucene.util.Bits;
 
@@ -103,7 +102,7 @@ public abstract class AbstractPrefixTree
       bitSet.or(wrap(postingsEnum, acceptDocs));
     }
 
-    protected void collectDocs(BitDocIdSet.Builder bitSetBuilder) throws IOException {
+    protected void collectDocs(BitDocIdSetBuilder bitSetBuilder) throws IOException {
       assert termsEnum != null;
       postingsEnum = termsEnum.postings(postingsEnum, PostingsEnum.NONE);
       bitSetBuilder.or(wrap(postingsEnum, acceptDocs));

Added: lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/util/BitDocIdSetBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/util/BitDocIdSetBuilder.java?rev=1690026&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/util/BitDocIdSetBuilder.java (added)
+++ lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/util/BitDocIdSetBuilder.java Thu Jul  9 08:39:00 2015
@@ -0,0 +1,119 @@
+package org.apache.lucene.spatial.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.io.IOException;
+
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BitDocIdSet;
+import org.apache.lucene.util.BitSetIterator;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.SparseFixedBitSet;
+
+/**
+ * A builder of {@link DocIdSet}s that supports random access.
+ * @lucene.internal
+ */
+public final class BitDocIdSetBuilder {
+
+  private final int maxDoc;
+  private final int threshold;
+  private SparseFixedBitSet sparseSet;
+  private FixedBitSet denseSet;
+
+  // we cache an upper bound of the cost of this builder so that we don't have
+  // to re-compute approximateCardinality on the sparse set every time
+  private long costUpperBound;
+
+  /** Create a new empty instance. */
+  public BitDocIdSetBuilder(int maxDoc) {
+    this.maxDoc = maxDoc;
+    threshold = maxDoc >>> 10;
+  }
+
+  // pkg-private for testing
+  boolean dense() {
+    return denseSet != null;
+  }
+
+  /**
+   * Is this builder definitely empty?  If so, {@link #build()} will return null.  This is usually the same as
+   * simply being empty but if this builder was constructed with the {@code full} option or if an iterator was passed
+   * that iterated over no documents, then we're not sure.
+   */
+  public boolean isDefinitelyEmpty() {
+    return sparseSet == null && denseSet == null;
+  }
+
+  /**
+   * Add the content of the provided {@link DocIdSetIterator} to this builder.
+   */
+  public void or(DocIdSetIterator it) throws IOException {
+    if (denseSet != null) {
+      // already upgraded
+      denseSet.or(it);
+      return;
+    }
+
+    final long itCost = it.cost();
+    costUpperBound += itCost;
+    if (costUpperBound >= threshold) {
+      costUpperBound = (sparseSet == null ? 0 : sparseSet.approximateCardinality()) + itCost;
+
+      if (costUpperBound >= threshold) {
+        // upgrade
+        denseSet = new FixedBitSet(maxDoc);
+        denseSet.or(it);
+        if (sparseSet != null) {
+          denseSet.or(new BitSetIterator(sparseSet, 0L));
+        }
+        return;
+      }
+    }
+
+    // we are still sparse
+    if (sparseSet == null) {
+      sparseSet = new SparseFixedBitSet(maxDoc);
+    }
+    sparseSet.or(it);
+  }
+
+  /**
+   * Build a {@link DocIdSet} that contains all doc ids that have been added.
+   * This method may return <tt>null</tt> if no documents were addded to this
+   * builder.
+   * NOTE: this is a destructive operation, the builder should not be used
+   * anymore after this method has been called.
+   */
+  public BitDocIdSet build() {
+    final BitDocIdSet result;
+    if (denseSet != null) {
+      result = new BitDocIdSet(denseSet);
+    } else if (sparseSet != null) {
+      result = new BitDocIdSet(sparseSet);
+    } else {
+      result = null;
+    }
+    denseSet = null;
+    sparseSet = null;
+    costUpperBound = 0;
+    return result;
+  }
+
+}

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java?rev=1690026&r1=1690025&r2=1690026&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java Thu Jul  9 08:39:00 2015
@@ -17,7 +17,6 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import java.io.IOException;
 import java.util.BitSet;
 import java.util.Random;
 
@@ -29,13 +28,10 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.BitDocIdSet;
-import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.apache.lucene.util.TestUtil;
@@ -145,7 +141,7 @@ public abstract class SearchEquivalenceT
   /**
    * Returns a random filter over the document set
    */
-  protected Filter randomFilter() {
+  protected Query randomFilter() {
     final Query query;
     if (random().nextBoolean()) {
       query = TermRangeQuery.newStringRange("field", "a", "" + randomChar(), true, true);
@@ -154,89 +150,7 @@ public abstract class SearchEquivalenceT
       PhraseQuery phrase = new PhraseQuery(100, "field", "" + randomChar(), "" + randomChar());
       query = phrase;
     }
-    
-    // now wrap the query as a filter. QWF has its own codepath
-    if (random().nextBoolean()) {
-      return new QueryWrapperFilter(query);
-    } else {
-      return new SlowWrapperFilter(query, random().nextBoolean());
-    }
-  }
-  
-  static class SlowWrapperFilter extends Filter {
-    final Query query;
-    final boolean useBits;
-    
-    SlowWrapperFilter(Query query, boolean useBits) {
-      this.query = query;
-      this.useBits = useBits;
-    }
-    
-    @Override
-    public Query rewrite(IndexReader reader) throws IOException {
-      Query q = query.rewrite(reader);
-      if (q != query) {
-        return new SlowWrapperFilter(q, useBits);
-      } else {
-        return this;
-      }
-    }
-
-    @Override
-    public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
-      // get a private context that is used to rewrite, createWeight and score eventually
-      final LeafReaderContext privateContext = context.reader().getContext();
-      final Weight weight = new IndexSearcher(privateContext).createNormalizedWeight(query, false);
-      return new DocIdSet() {
-        @Override
-        public DocIdSetIterator iterator() throws IOException {
-          return weight.scorer(privateContext);
-        }
-
-        @Override
-        public long ramBytesUsed() {
-          return 0L;
-        }
-
-        @Override
-        public Bits bits() throws IOException {
-          if (useBits) {
-            BitDocIdSet.Builder builder = new BitDocIdSet.Builder(context.reader().maxDoc());
-            DocIdSetIterator disi = iterator();
-            if (disi != null) {
-              builder.or(disi);
-            }
-            BitDocIdSet bitset = builder.build();
-            if (bitset == null) {
-              return new Bits.MatchNoBits(context.reader().maxDoc());
-            } else {
-              return bitset.bits();
-            }
-          } else {
-            return null;
-          }
-        }
-      };
-    }
-
-    @Override
-    public String toString(String field) {
-      return "SlowQWF(" + query + ")";
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      if (super.equals(obj) == false) {
-        return false;
-      }
-      return query.equals(((SlowWrapperFilter) obj).query);
-    }
-
-    @Override
-    public int hashCode() {
-      return 31 * super.hashCode() + query.hashCode();
-    }
-
+    return query;
   }
 
   /**
@@ -259,13 +173,10 @@ public abstract class SearchEquivalenceT
     // test with some filters (this will sometimes cause advance'ing enough to test it)
     int numFilters = TEST_NIGHTLY ? atLeast(10) : atLeast(3);
     for (int i = 0; i < numFilters; i++) {
-      Filter filter = randomFilter();
+      Query filter = randomFilter();
       // incorporate the filter in different ways.
       assertSubsetOf(q1, q2, filter);
       assertSubsetOf(filteredQuery(q1, filter), filteredQuery(q2, filter), null);
-      assertSubsetOf(filteredQuery(q1, filter), filteredBooleanQuery(q2, filter), null);
-      assertSubsetOf(filteredBooleanQuery(q1, filter), filteredBooleanQuery(q2, filter), null);
-      assertSubsetOf(filteredBooleanQuery(q1, filter), filteredQuery(q2, filter), null);
     }
   }
   
@@ -319,13 +230,10 @@ public abstract class SearchEquivalenceT
     // also test with some filters to test advancing
     int numFilters = TEST_NIGHTLY ? atLeast(10) : atLeast(3);
     for (int i = 0; i < numFilters; i++) {
-      Filter filter = randomFilter();
+      Query filter = randomFilter();
       // incorporate the filter in different ways.
       assertSameScores(q1, q2, filter);
       assertSameScores(filteredQuery(q1, filter), filteredQuery(q2, filter), null);
-      assertSameScores(filteredQuery(q1, filter), filteredBooleanQuery(q2, filter), null);
-      assertSameScores(filteredBooleanQuery(q1, filter), filteredBooleanQuery(q2, filter), null);
-      assertSameScores(filteredBooleanQuery(q1, filter), filteredQuery(q2, filter), null);
     }
   }
 
@@ -356,11 +264,4 @@ public abstract class SearchEquivalenceT
         .add(filter, Occur.FILTER)
         .build();
   }
-  
-  protected Query filteredBooleanQuery(Query query, Filter filter) {
-    BooleanQuery.Builder bq = new BooleanQuery.Builder();
-    bq.add(query, Occur.MUST);
-    bq.add(filter, Occur.FILTER);
-    return bq.build();
-  }
 }