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 2014/10/24 09:32:20 UTC

svn commit: r1634012 [1/2] - in /lucene/dev/trunk: lucene/ lucene/codecs/src/java/org/apache/lucene/codecs/bloom/ lucene/core/src/java/org/apache/lucene/search/ lucene/core/src/java/org/apache/lucene/util/ lucene/core/src/test/org/apache/lucene/search/...

Author: jpountz
Date: Fri Oct 24 07:32:19 2014
New Revision: 1634012

URL: http://svn.apache.org/r1634012
Log:
LUCENE-6021: Make SparseFixedBitSet and FixedBitSet share a common "BitSet" interface.

Added:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BitDocIdSet.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BitSet.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BitSetIterator.java   (with props)
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestFixedBitDocIdSet.java   (with props)
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestSparseFixedBitDocIdSet.java   (with props)
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/BaseBitSetTestCase.java   (with props)
Removed:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/FixedBitDocIdSet.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/SparseFixedBitDocIdSet.java
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/FuzzySet.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesDocIdSet.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/RoaringDocIdSet.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/SparseFixedBitSet.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/MockFilter.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/SingleDocTestFilter.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestCachingWrapperFilter.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilteredQuery.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilteredSearch.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestSortRandom.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestFixedBitSet.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestNotDocIdSet.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestSparseFixedBitSet.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/RandomSamplingFacetsCollector.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
    lucene/dev/trunk/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java
    lucene/dev/trunk/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterPhraseTest.java
    lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/FixedBitSetCachingWrapperFilter.java
    lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java
    lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java
    lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinFieldComparator.java
    lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java
    lucene/dev/trunk/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
    lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/index/PKIndexSplitter.java
    lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/index/sorter/BlockJoinComparatorSource.java
    lucene/dev/trunk/lucene/misc/src/test/org/apache/lucene/index/sorter/TestBlockJoinSorter.java
    lucene/dev/trunk/lucene/misc/src/test/org/apache/lucene/uninverting/TestFieldCacheSortRandom.java
    lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/BooleanFilter.java
    lucene/dev/trunk/lucene/queries/src/test/org/apache/lucene/queries/BooleanFilterTest.java
    lucene/dev/trunk/lucene/queries/src/test/org/apache/lucene/queries/TermsFilterTest.java
    lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/DuplicateFilter.java
    lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/search/TestTermAutomatonQuery.java
    lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/IntersectsPrefixTreeFilter.java
    lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/WithinPrefixTreeFilter.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/BaseDocIdSetTestCase.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/response/SortingResponseWriter.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/BitDocSet.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/DocSetBase.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/join/BitSetSlice.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestDocSet.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestFiltering.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestSort.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/util/BitSetPerf.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Fri Oct 24 07:32:19 2014
@@ -171,6 +171,9 @@ API Changes
   FieldInfo, since it's redundant with IndexOptions != null. (Robert
   Muir, Mike McCandless)
 
+* LUCENE-6021: FixedBitSet.nextSetBit now returns DocIdSetIterator.NO_MORE_DOCS
+  instead of -1 when there are no more bits which are set. (Adrien Grand)
+
 Bug Fixes
 
 * LUCENE-5650: Enforce read-only access to any path outside the temporary

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/FuzzySet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/FuzzySet.java?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/FuzzySet.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/FuzzySet.java Fri Oct 24 07:32:19 2014
@@ -19,6 +19,7 @@ package org.apache.lucene.codecs.bloom;
 import java.io.IOException;
 import java.util.Collections;
 
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.util.Accountable;
@@ -272,7 +273,7 @@ public class FuzzySet implements Account
       int bitIndex = 0;
       do {
         bitIndex = filter.nextSetBit(bitIndex);
-        if (bitIndex >= 0) {
+        if (bitIndex != DocIdSetIterator.NO_MORE_DOCS) {
           // Project the larger number into a smaller one effectively
           // modulo-ing by using the target bitset size as a mask
           int downSizedBitIndex = bitIndex & rightSizedBitSetSize;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesDocIdSet.java?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesDocIdSet.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesDocIdSet.java Fri Oct 24 07:32:19 2014
@@ -19,7 +19,7 @@ package org.apache.lucene.search;
 import java.io.IOException;
 
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.FixedBitDocIdSet;
+import org.apache.lucene.util.BitDocIdSet;
 import org.apache.lucene.util.FixedBitSet;
 
 /**
@@ -116,7 +116,7 @@ public abstract class DocValuesDocIdSet 
     } else if (acceptDocs instanceof FixedBitSet) {
       // special case for FixedBitSet: use the iterator and filter it
       // (used e.g. when Filters are chained by FilteredQuery)
-      return new FilteredDocIdSetIterator(new FixedBitDocIdSet((FixedBitSet) acceptDocs).iterator()) {
+      return new FilteredDocIdSetIterator(new BitDocIdSet((FixedBitSet) acceptDocs).iterator()) {
         @Override
         protected boolean match(int doc) {
           return DocValuesDocIdSet.this.matchDoc(doc);

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java Fri Oct 24 07:32:19 2014
@@ -511,7 +511,7 @@ final class SloppyPhraseScorer extends S
     Term[] t = tord.keySet().toArray(new Term[0]);
     for (int i=0; i<bb.size(); i++) { // i is the group no.
       FixedBitSet bits = bb.get(i);
-      for (int ord = bits.nextSetBit(0); ord != -1; ord = ord + 1 >= bits.length() ? -1 : bits.nextSetBit(ord + 1)) {
+      for (int ord = bits.nextSetBit(0); ord != DocIdSetIterator.NO_MORE_DOCS; ord = ord + 1 >= bits.length() ? DocIdSetIterator.NO_MORE_DOCS : bits.nextSetBit(ord + 1)) {
         tg.put(t[ord],i);
       }
     }

Added: 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=1634012&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BitDocIdSet.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BitDocIdSet.java Fri Oct 24 07:32:19 2014
@@ -0,0 +1,78 @@
+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 org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.DocIdSetIterator;
+
+/**
+ * Implementation of the {@link DocIdSet} interface on top of a {@link BitSet}.
+ * @lucene.internal
+ */
+public class BitDocIdSet extends DocIdSet {
+
+  private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(BitDocIdSet.class);
+
+  private final BitSet set;
+  private final long cost;
+
+  /**
+   * Wrap the given {@link FixedBitSet} as a {@link DocIdSet}. The provided
+   * {@link FixedBitSet} should not be modified after having wrapped as a
+   * {@link DocIdSet}.
+   */
+  public BitDocIdSet(BitSet set, long cost) {
+    this.set = set;
+    this.cost = cost;
+  }
+
+  /**
+   * Same as {@link #BitDocIdSet(BitSet, long)} but uses the set's
+   * {@link BitSet#approximateCardinality() approximate cardinality} as a cost.
+   */
+  public BitDocIdSet(BitSet set) {
+    this(set, set.approximateCardinality());
+  }
+
+  @Override
+  public DocIdSetIterator iterator() {
+    return new BitSetIterator(set, cost);
+  }
+
+  @Override
+  public BitSet bits() {
+    return set;
+  }
+
+  /** This DocIdSet implementation is cacheable. */
+  @Override
+  public boolean isCacheable() {
+    return true;
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return BASE_RAM_BYTES_USED + set.ramBytesUsed();
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(set=" + set + ",cost=" + cost + ")";
+  }
+
+}

Added: 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=1634012&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BitSet.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BitSet.java Fri Oct 24 07:32:19 2014
@@ -0,0 +1,94 @@
+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.DocIdSetIterator;
+
+/**
+ * Base implementation for a bit set.
+ * @lucene.internal
+ */
+public abstract class BitSet implements MutableBits, Accountable {
+
+  /** Set the bit at <code>i</code>. */
+  public abstract void set(int i);
+
+  /** Clears a range of bits.
+   *
+   * @param startIndex lower index
+   * @param endIndex one-past the last bit to clear
+   */
+  public abstract void clear(int startIndex, int endIndex);
+
+  /**
+   * Return the number of bits that are set.
+   * NOTE: this method is likely to run in linear time
+   */
+  public abstract int cardinality();
+
+  /**
+   * Return an approximation of the cardinality of this set. Some
+   * implementations may trade accuracy for speed if they have the ability to
+   * estimate the cardinality of the set without iterating over all the data.
+   * The default implementation returns {@link #cardinality()}.
+   */
+  public int approximateCardinality() {
+    return cardinality();
+  }
+
+  /** Returns the index of the first set bit starting at the index specified.
+   *  {@link DocIdSetIterator#NO_MORE_DOCS} is returned if there are no more set bits.
+   */
+  public abstract int nextSetBit(int i);
+
+  /** Does in-place OR of the bits provided by the
+   *  iterator. */
+  public void or(DocIdSetIterator iter) throws IOException {
+    for (int doc = iter.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = iter.nextDoc()) {
+      set(doc);
+    }
+  }
+
+  /** Does in-place AND of the bits provided by the
+   *  iterator. */
+  public void and(DocIdSetIterator iter) throws IOException {
+    final int length = length();
+    if (length == 0) {
+      return;
+    }
+    int disiDoc, bitSetDoc = nextSetBit(0);
+    while (bitSetDoc != DocIdSetIterator.NO_MORE_DOCS && (disiDoc = iter.advance(bitSetDoc)) < length) {
+      clear(bitSetDoc, disiDoc);
+      disiDoc++;
+      bitSetDoc = (disiDoc < length) ? nextSetBit(disiDoc) : DocIdSetIterator.NO_MORE_DOCS;
+    }
+    if (bitSetDoc != DocIdSetIterator.NO_MORE_DOCS) {
+      clear(bitSetDoc, length);
+    }
+  }
+
+  /** this = this AND NOT other */
+  public void andNot(DocIdSetIterator iter) throws IOException {
+    for (int doc = iter.nextDoc(), len = length(); doc < len; doc = iter.nextDoc()) {
+      clear(doc);
+    }
+  }
+
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BitSetIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BitSetIterator.java?rev=1634012&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BitSetIterator.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BitSetIterator.java Fri Oct 24 07:32:19 2014
@@ -0,0 +1,80 @@
+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 org.apache.lucene.search.DocIdSetIterator;
+
+/**
+ * A {@link DocIdSetIterator} which iterates over set bits in a
+ * bit set.
+ * @lucene.internal
+ */
+public class BitSetIterator extends DocIdSetIterator {
+
+  private static <T extends BitSet> T getBitSet(DocIdSetIterator iterator, Class<? extends T> clazz) {
+    if (iterator instanceof BitSetIterator) {
+      BitSet bits = ((BitSetIterator) iterator).bits;
+      assert bits != null;
+      if (clazz.isInstance(bits)) {
+        return clazz.cast(bits);
+      }
+    }
+    return null;
+  }
+
+  /** If the provided iterator wraps a {@link FixedBitSet}, returns it, otherwise returns null. */
+  public static FixedBitSet getFixedBitSetOrNull(DocIdSetIterator iterator) {
+    return getBitSet(iterator, FixedBitSet.class);
+  }
+
+  private final BitSet bits;
+  private final int length;
+  private final long cost;
+  private int doc = -1;
+
+  /** Sole constructor. */
+  public BitSetIterator(BitSet bits, long cost) {
+    this.bits = bits;
+    this.length = bits.length();
+    this.cost = cost;
+  }
+
+  @Override
+  public int docID() {
+    return doc;
+  }
+
+  @Override
+  public int nextDoc() {
+    return advance(doc + 1);
+  }
+
+  @Override
+  public int advance(int target) {
+    if (target >= length) {
+      return doc = NO_MORE_DOCS;
+    }
+    return doc = bits.nextSetBit(target);
+  }
+
+  @Override
+  public long cost() {
+    return cost;
+  }
+
+}

Modified: 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=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/DocIdSetBuilder.java Fri Oct 24 07:32:19 2014
@@ -21,7 +21,6 @@ import java.io.IOException;
 
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.util.SparseFixedBitSet.SparseFixedBitSetIterator;
 
 /**
  * A builder of {@link DocIdSet}s that supports random access.
@@ -38,10 +37,19 @@ public final class DocIdSetBuilder {
   // to re-compute approximateCardinality on the sparse set every time 
   private long costUpperBound;
 
-  /** Sole constructor. */
-  public DocIdSetBuilder(int maxDoc) {
+  /** Create a new instance that can hold <code>maxDoc</code> documents and is optionally <code>full</code>. */
+  public DocIdSetBuilder(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 DocIdSetBuilder(int maxDoc) {
+    this(maxDoc, false);
   }
 
   /**
@@ -64,7 +72,7 @@ public final class DocIdSetBuilder {
         denseSet = new FixedBitSet(maxDoc);
         denseSet.or(it);
         if (sparseSet != null) {
-          denseSet.or(new SparseFixedBitSetIterator(sparseSet, 0L));
+          denseSet.or(new BitSetIterator(sparseSet, 0L));
         }
         return;
       }
@@ -78,6 +86,28 @@ public final class DocIdSetBuilder {
   }
 
   /**
+   * Removes from this builder documents that are not contained in <code>it</code>.
+   */
+  public void and(DocIdSetIterator it) throws IOException {
+    if (denseSet != null) {
+      denseSet.and(it);
+    } else if (sparseSet != null) {
+      sparseSet.and(it);
+    }
+  }
+
+  /**
+   * Removes from this builder documents that are contained in <code>it</code>.
+   */
+  public void andNot(DocIdSetIterator it) throws IOException {
+    if (denseSet != null) {
+      denseSet.andNot(it);
+    } else if (denseSet != null) {
+      denseSet.andNot(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.
@@ -87,9 +117,9 @@ public final class DocIdSetBuilder {
   public DocIdSet build() {
     final DocIdSet result;
     if (denseSet != null) {
-      result = new FixedBitDocIdSet(denseSet, denseSet.cardinality());
+      result = new BitDocIdSet(denseSet);
     } else if (sparseSet != null) {
-      result = new SparseFixedBitDocIdSet(sparseSet, sparseSet.approximateCardinality());
+      result = new BitDocIdSet(sparseSet);
     } else {
       result = null;
     }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java Fri Oct 24 07:32:19 2014
@@ -32,73 +32,11 @@ import org.apache.lucene.search.DocIdSet
  * 
  * @lucene.internal
  */
-public final class FixedBitSet implements MutableBits, Accountable {
+public final class FixedBitSet extends BitSet implements MutableBits, Accountable {
 
   private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(FixedBitSet.class);
 
   /**
-   * A {@link DocIdSetIterator} which iterates over set bits in a
-   * {@link FixedBitSet}.
-   */
-  public static final class FixedBitSetIterator extends DocIdSetIterator {
-    
-    final int numBits, numWords;
-    final long[] bits;
-    final long cost;
-    int doc = -1;
-    
-    /** Creates an iterator over the given {@link FixedBitSet}. */
-    public FixedBitSetIterator(FixedBitSet bits, long cost) {
-      this(bits.bits, bits.numBits, bits.numWords, cost);
-    }
-    
-    /** Creates an iterator over the given array of bits. */
-    public FixedBitSetIterator(long[] bits, int numBits, int wordLength, long cost) {
-      this.bits = bits;
-      this.numBits = numBits;
-      this.numWords = wordLength;
-      this.cost = cost;
-    }
-    
-    @Override
-    public int nextDoc() {
-      return advance(doc + 1);
-    }
-    
-    @Override
-    public int docID() {
-      return doc;
-    }
-    
-    @Override
-    public long cost() {
-      return cost;
-    }
-    
-    @Override
-    public int advance(int target) {
-      if (target >= numBits) {
-        return doc = NO_MORE_DOCS;
-      }
-      int i = target >> 6;
-      long word = bits[i] >> target; // skip all the bits to the right of index
-      
-      if (word != 0) {
-        return doc = target + Long.numberOfTrailingZeros(word);
-      }
-      
-      while (++i < numWords) {
-        word = bits[i];
-        if (word != 0) {
-          return doc = (i << 6) + Long.numberOfTrailingZeros(word);
-        }
-      }
-      
-      return doc = NO_MORE_DOCS;
-    }
-  }
-
-  /**
    * If the given {@link FixedBitSet} is large enough to hold {@code numBits},
    * returns the given bits, otherwise returns a new {@link FixedBitSet} which
    * can hold the requested number of bits.
@@ -203,9 +141,7 @@ public final class FixedBitSet implement
     return bits;
   }
 
-  /** Returns number of set bits.  NOTE: this visits every
-   *  long in the backing bits array, and the result is not
-   *  internally cached! */
+  @Override
   public int cardinality() {
     return (int) BitUtil.pop_array(bits, 0, bits.length);
   }
@@ -236,6 +172,7 @@ public final class FixedBitSet implement
     return val;
   }
 
+  @Override
   public void clear(int index) {
     assert index >= 0 && index < numBits;
     int wordNum = index >> 6;
@@ -252,9 +189,7 @@ public final class FixedBitSet implement
     return val;
   }
 
-  /** Returns the index of the first set bit starting at the index specified.
-   *  -1 is returned if there are no more set bits.
-   */
+  @Override
   public int nextSetBit(int index) {
     assert index >= 0 && index < numBits : "index=" + index + ", numBits=" + numBits;
     int i = index >> 6;
@@ -271,7 +206,7 @@ public final class FixedBitSet implement
       }
     }
 
-    return -1;
+    return DocIdSetIterator.NO_MORE_DOCS;
   }
 
   /** Returns the index of the last set bit before or on the index specified.
@@ -297,20 +232,16 @@ public final class FixedBitSet implement
     return -1;
   }
 
-  /** Does in-place OR of the bits provided by the
-   *  iterator. */
+  @Override
   public void or(DocIdSetIterator iter) throws IOException {
-    if (iter instanceof FixedBitSetIterator && iter.docID() == -1) {
-      final FixedBitSetIterator fbs = (FixedBitSetIterator) iter;
-      or(fbs.bits, fbs.numWords);
+    if (BitSetIterator.getFixedBitSetOrNull(iter) != null && iter.docID() == -1) {
+      final FixedBitSet bits = BitSetIterator.getFixedBitSetOrNull(iter); 
+      or(bits);
       // advance after last doc that would be accepted if standard
       // iteration is used (to exhaust it):
-      fbs.advance(numBits);
+      iter.advance(numBits);
     } else {
-      int doc;
-      while ((doc = iter.nextDoc()) < numBits) {
-        set(doc);
-      }
+      super.or(iter);
     }
   }
 
@@ -335,12 +266,12 @@ public final class FixedBitSet implement
   
   /** Does in-place XOR of the bits provided by the iterator. */
   public void xor(DocIdSetIterator iter) throws IOException {
-    if (iter instanceof FixedBitSetIterator && iter.docID() == -1) {
-      final FixedBitSetIterator fbs = (FixedBitSetIterator) iter;
-      xor(fbs.bits, fbs.numWords);
+    if (BitSetIterator.getFixedBitSetOrNull(iter) != null && iter.docID() == -1) {
+      final FixedBitSet bits = BitSetIterator.getFixedBitSetOrNull(iter); 
+      xor(bits);
       // advance after last doc that would be accepted if standard
       // iteration is used (to exhaust it):
-      fbs.advance(numBits);
+      iter.advance(numBits);
     } else {
       int doc;
       while ((doc = iter.nextDoc()) < numBits) {
@@ -358,26 +289,16 @@ public final class FixedBitSet implement
     }
   }
 
-  /** Does in-place AND of the bits provided by the
-   *  iterator. */
+  @Override
   public void and(DocIdSetIterator iter) throws IOException {
-    if (iter instanceof FixedBitSetIterator && iter.docID() == -1) {
-      final FixedBitSetIterator fbs = (FixedBitSetIterator) iter;
-      and(fbs.bits, fbs.numWords);
+    if (BitSetIterator.getFixedBitSetOrNull(iter) != null && iter.docID() == -1) {
+      final FixedBitSet bits = BitSetIterator.getFixedBitSetOrNull(iter); 
+      and(bits);
       // advance after last doc that would be accepted if standard
       // iteration is used (to exhaust it):
-      fbs.advance(numBits);
+      iter.advance(numBits);
     } else {
-      if (numBits == 0) return;
-      int disiDoc, bitSetDoc = nextSetBit(0);
-      while (bitSetDoc != -1 && (disiDoc = iter.advance(bitSetDoc)) < numBits) {
-        clear(bitSetDoc, disiDoc);
-        disiDoc++;
-        bitSetDoc = (disiDoc < numBits) ? nextSetBit(disiDoc) : -1;
-      }
-      if (bitSetDoc != -1) {
-        clear(bitSetDoc, numBits);
-      }
+      super.and(iter);
     }
   }
 
@@ -406,20 +327,16 @@ public final class FixedBitSet implement
     }
   }
 
-  /** Does in-place AND NOT of the bits provided by the
-   *  iterator. */
+  @Override
   public void andNot(DocIdSetIterator iter) throws IOException {
-    if (iter instanceof FixedBitSetIterator && iter.docID() == -1) {
-      final FixedBitSetIterator fbs = (FixedBitSetIterator) iter;
-      andNot(fbs.bits, fbs.numWords);
+    if (BitSetIterator.getFixedBitSetOrNull(iter) != null && iter.docID() == -1) {
+      final FixedBitSet bits = BitSetIterator.getFixedBitSetOrNull(iter); 
+      andNot(bits);
       // advance after last doc that would be accepted if standard
       // iteration is used (to exhaust it):
-      fbs.advance(numBits);
+      iter.advance(numBits);
     } else {
-      int doc;
-      while ((doc = iter.nextDoc()) < numBits) {
-        clear(doc);
-      }
+      super.andNot(iter);
     }
   }
 
@@ -516,11 +433,7 @@ public final class FixedBitSet implement
     bits[endWord] |= endmask;
   }
 
-  /** Clears a range of bits.
-   *
-   * @param startIndex lower index
-   * @param endIndex one-past the last bit to clear
-   */
+  @Override
   public void clear(int startIndex, int endIndex) {
     assert startIndex >= 0 && startIndex < numBits : "startIndex=" + startIndex + ", numBits=" + numBits;
     assert endIndex >= 0 && endIndex <= numBits : "endIndex=" + endIndex + ", numBits=" + numBits;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/RoaringDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/RoaringDocIdSet.java?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/RoaringDocIdSet.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/RoaringDocIdSet.java Fri Oct 24 07:32:19 2014
@@ -85,14 +85,14 @@ public class RoaringDocIdSet extends Doc
           int excludedDoc = -1;
           for (int i = 0; i < excludedDocs.length; ++i) {
             excludedDoc = denseBuffer.nextSetBit(excludedDoc + 1);
-            assert excludedDoc != -1;
+            assert excludedDoc != DocIdSetIterator.NO_MORE_DOCS;
             excludedDocs[i] = (short) excludedDoc;
           }
-          assert excludedDoc + 1 == denseBuffer.length() || denseBuffer.nextSetBit(excludedDoc + 1) == -1;
+          assert excludedDoc + 1 == denseBuffer.length() || denseBuffer.nextSetBit(excludedDoc + 1) == DocIdSetIterator.NO_MORE_DOCS;
           sets[currentBlock] = new NotDocIdSet(BLOCK_SIZE, new ShortArrayDocIdSet(excludedDocs));
         } else {
           // Neither sparse nor super dense, use a fixed bit set
-          sets[currentBlock] = new FixedBitDocIdSet(denseBuffer, currentBlockCardinality);
+          sets[currentBlock] = new BitDocIdSet(denseBuffer, currentBlockCardinality);
         }
         denseBuffer = null;
       }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/SparseFixedBitSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/SparseFixedBitSet.java?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/SparseFixedBitSet.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/SparseFixedBitSet.java Fri Oct 24 07:32:19 2014
@@ -33,10 +33,11 @@ import org.apache.lucene.search.DocIdSet
  *
  * @lucene.internal
  */
-public class SparseFixedBitSet implements Bits, Accountable {
+public class SparseFixedBitSet extends BitSet implements Bits, Accountable {
 
   private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(SparseFixedBitSet.class);
   private static final long SINGLE_ELEMENT_ARRAY_BYTES_USED = RamUsageEstimator.sizeOf(new long[1]);
+  private static final int MASK_4096 = (1 << 12) - 1;
 
   private static int blockCount(int length) {
     int blockCount = length >>> 12;
@@ -78,10 +79,7 @@ public class SparseFixedBitSet implement
     return true;
   }
 
-  /**
-   * Compute the cardinality of this set.
-   * NOTE: this operation runs in linear time.
-   */
+  @Override
   public int cardinality() {
     int cardinality = 0;
     for (long[] bitArray : bits) {
@@ -94,12 +92,11 @@ public class SparseFixedBitSet implement
     return cardinality;
   }
 
-  /**
-   * Return an approximation of the cardinality of this set, assuming that bits
-   * are uniformly distributed. This operation runs in constant time.
-   */
+  @Override
   public int approximateCardinality() {
-    // this is basically the linear counting algorithm
+    // we are assuming that bits are uniformly set and use the linear counting
+    // algorithm to estimate the number of bits that are set based on the number
+    // of longs that are different from zero
     final int totalLongs = (length + 63) >>> 6; // total number of longs in the space
     assert totalLongs >= nonZeroLongCount;
     final int zeroLongs = totalLongs - nonZeroLongCount; // number of longs that are zeros
@@ -193,134 +190,173 @@ public class SparseFixedBitSet implement
   }
 
   /**
-   * Add the documents contained in the provided {@link DocIdSetIterator} to
-   * this bit set.
+   * Clear the bit at index <tt>i</tt>.
    */
-  public void or(DocIdSetIterator it) throws IOException {
-    for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
-      set(doc);
+  public void clear(int i) {
+    assert consistent(i);
+    final int i4096 = i >>> 12;
+    final int i64 = i >>> 6;
+    clearWithinLong(i4096, i64, ~(1L << i));
+  }
+
+  private void clearWithinLong(int i4096, int i64, long mask) {
+    final long index = indices[i4096];
+    if ((index & (1L << i64)) != 0) {
+      // offset of the long bits we are interested in in the array
+      final int o = Long.bitCount(index & ((1L << i64) - 1));
+      long bits = this.bits[i4096][o] & mask;
+      if (bits == 0) {
+        removeLong(i4096, i64, index, o);
+      } else {
+        this.bits[i4096][o] = bits;
+      }
     }
   }
 
-  @Override
-  public long ramBytesUsed() {
-    return ramBytesUsed;
+  private void removeLong(int i4096, int i64, long index, int o) {
+    index &= ~(1L << i64);
+    indices[i4096] = index;
+    if (index == 0) {
+      // release memory, there is nothing in this block anymore
+      this.bits[i4096] = null;
+    } else {
+      final int length = Long.bitCount(index);
+      final long[] bitArray = bits[i4096];
+      System.arraycopy(bitArray, o + 1, bitArray, o, length - o);
+      bitArray[length] = 0L;
+    }
   }
 
   @Override
-  public Iterable<? extends Accountable> getChildResources() {
-    return Collections.emptyList();
+  public void clear(int from, int to) {
+    assert from >= 0;
+    assert to <= length;
+    if (from >= to) {
+      return;
+    }
+    final int firstBlock = from >>> 12;
+    final int lastBlock = (to - 1) >>> 12;
+    if (firstBlock == lastBlock) {
+      clearWithinBlock(firstBlock, from & MASK_4096, (to - 1) & MASK_4096);
+    } else {
+      clearWithinBlock(firstBlock, from & MASK_4096, MASK_4096);
+      for (int i = firstBlock + 1; i < lastBlock; ++i) {
+        indices[i] = 0;
+        bits[i] = null;
+      }
+      clearWithinBlock(lastBlock, 0, (to - 1) & MASK_4096);
+    }
   }
 
-  /**
-   * A {@link DocIdSetIterator} which iterates over set bits in a
-   * {@link SparseFixedBitSet}.
-   */
-  public static class SparseFixedBitSetIterator extends DocIdSetIterator {
+  // create a long that has bits set to one between from and to
+  private static long mask(int from, int to) {
+    return ((1L << (to - from) << 1) - 1) << from;
+  }
 
-    private final long[] indices;
-    private final long[][] bits;
-    private final long cost;
-    private int doc = -1;
-
-    /** Sole constructor. */
-    public SparseFixedBitSetIterator(SparseFixedBitSet set, long cost) {
-      indices = set.indices;
-      bits = set.bits;
-      this.cost = cost;
-    }
-
-    @Override
-    public int docID() {
-      return doc;
-    }
-
-    /** Return the first document that occurs on or after the provided block index. */
-    private int firstDoc(int i4096) {
-      long index = 0;
-      while (i4096 < indices.length) {
-        index = indices[i4096];
-        if (index != 0) {
-          final int i64 = Long.numberOfTrailingZeros(index);
-          return doc = (i4096 << 12) | (i64 << 6) | Long.numberOfTrailingZeros(bits[i4096][0]);
-        }
-        i4096 += 1;
+  private void clearWithinBlock(int i4096, int from, int to) {
+    int firstLong = from >>> 6;
+    int lastLong = to >>> 6;
+
+    if (firstLong == lastLong) {
+      clearWithinLong(i4096, firstLong, ~mask(from, to));
+    } else {
+      assert firstLong < lastLong;
+      clearWithinLong(i4096, lastLong, ~mask(0, to));
+      for (int i = lastLong - 1; i >= firstLong + 1; --i) {
+        clearWithinLong(i4096, i, 0L);
       }
-      return doc = NO_MORE_DOCS;
+      clearWithinLong(i4096, firstLong, ~mask(from, 63));
     }
+  }
 
-    @Override
-    public int nextDoc() throws IOException {
-      return advance(doc + 1);
+  /** Return the first document that occurs on or after the provided block index. */
+  private int firstDoc(int i4096) {
+    long index = 0;
+    while (i4096 < indices.length) {
+      index = indices[i4096];
+      if (index != 0) {
+        final int i64 = Long.numberOfTrailingZeros(index);
+        return (i4096 << 12) | (i64 << 6) | Long.numberOfTrailingZeros(bits[i4096][0]);
+      }
+      i4096 += 1;
     }
+    return DocIdSetIterator.NO_MORE_DOCS;
+  }
 
-    @Override
-    public int advance(int target) throws IOException {
-      final int i4096 = target >>> 12;
-      if (i4096 >= indices.length) {
-        return doc = NO_MORE_DOCS;
+  @Override
+  public int nextSetBit(int i) {
+    assert i < length;
+    final int i4096 = i >>> 12;
+    final long index = indices[i4096];
+    int i64 = i >>> 6;
+    long indexBits = index >>> i64;
+    if (indexBits == 0) {
+      // if the index is zero, it means that there is no value in the
+      // current block, so return the first document of the next block
+      // or
+      // if neither the i64-th bit or any other bit on its left is set then
+      // it means that there are no more documents in this block, go to the
+      // next one
+      return firstDoc(i4096 + 1);
+    } else {
+      // We know we still have some 64-bits blocks that have bits set, let's
+      // advance to the next one by skipping trailing zeros of the index
+      int i1 = i & 0x3F;
+      int trailingZeros = Long.numberOfTrailingZeros(indexBits);
+      if (trailingZeros != 0) {
+        // no bits in the current long, go to the next one
+        i64 += trailingZeros;
+        i1 = 0;
       }
-      final long index = indices[i4096];
-      int i64 = target >>> 6;
-      long indexBits = index >>> i64;
-      if (indexBits == 0) {
-        // if the index is zero, it means that there is no value in the
-        // current block, so return the first document of the next block
-        // or
-        // if neither the i64-th bit or any other bit on its left is set then
-        // it means that there are no more documents in this block, go to the
-        // next one
-        return firstDoc(i4096 + 1);
-      } else {
-        // We know we still have some 64-bits blocks that have bits set, let's
-        // advance to the next one by skipping trailing zeros of the index
-        int i1 = target & 0x3F;
-        int trailingZeros = Long.numberOfTrailingZeros(indexBits);
-        if (trailingZeros != 0) {
-          // no bits in the current long, go to the next one
-          i64 += trailingZeros;
-          i1 = 0;
-        }
 
-        // So now we are on a sub 64-bits block that has values
-        assert (index & (1L << i64)) != 0;
-        // we count the number of ones on the left of i64 to figure out the
-        // index of the long that contains the bits we are interested in
-        int longIndex = Long.bitCount(index & ((1L << i64) - 1)); // shifts are mod 64 in java
-        final long[] longArray = bits[i4096];
-        assert longArray[longIndex] != 0;
-        long bits = longArray[longIndex] >>> i1; // shifts are mod 64 in java
-        if (bits != 0L) {
-          // hurray, we found some non-zero bits, this gives us the next document:
-          i1 += Long.numberOfTrailingZeros(bits);
-          return doc = (i4096 << 12) | ((i64 & 0x3F) << 6) | i1;
-        }
+      // So now we are on a sub 64-bits block that has values
+      assert (index & (1L << i64)) != 0;
+      // we count the number of ones on the left of i64 to figure out the
+      // index of the long that contains the bits we are interested in
+      int longIndex = Long.bitCount(index & ((1L << i64) - 1)); // shifts are mod 64 in java
+      final long[] longArray = bits[i4096];
+      assert longArray[longIndex] != 0;
+      long bits = longArray[longIndex] >>> i1; // shifts are mod 64 in java
+      if (bits != 0L) {
+        // hurray, we found some non-zero bits, this gives us the next document:
+        i1 += Long.numberOfTrailingZeros(bits);
+        return (i4096 << 12) | ((i64 & 0x3F) << 6) | i1;
+      }
 
-        // otherwise it means that although we were on a sub-64 block that contains
-        // documents, all documents of this sub-block have already been consumed
-        // so two cases:
-        indexBits = index >>> i64 >>> 1; // we don't shift by (i64+1) otherwise we might shift by a multiple of 64 which is a no-op
-        if (indexBits == 0) {
-          // Case 1: this was the last long of the block of 4096 bits, then go
-          // to the next block
-          return firstDoc(i4096 + 1);
-        }
-        // Case 2: go to the next sub 64-bits block in the current block of 4096 bits
-        // by skipping trailing zeros of the index
-        trailingZeros = Long.numberOfTrailingZeros(indexBits);
-        i64 += 1 + trailingZeros;
-        bits = longArray[longIndex + 1];
-        assert bits != 0;
-        i1 = Long.numberOfTrailingZeros(bits);
-        return doc = (i4096 << 12) | ((i64 & 0x3F) << 6) | i1;
+      // otherwise it means that although we were on a sub-64 block that contains
+      // documents, all documents of this sub-block have already been consumed
+      // so two cases:
+      indexBits = index >>> i64 >>> 1; // we don't shift by (i64+1) otherwise we might shift by a multiple of 64 which is a no-op
+      if (indexBits == 0) {
+        // Case 1: this was the last long of the block of 4096 bits, then go
+        // to the next block
+        return firstDoc(i4096 + 1);
       }
+      // Case 2: go to the next sub 64-bits block in the current block of 4096 bits
+      // by skipping trailing zeros of the index
+      trailingZeros = Long.numberOfTrailingZeros(indexBits);
+      i64 += 1 + trailingZeros;
+      bits = longArray[longIndex + 1];
+      assert bits != 0;
+      i1 = Long.numberOfTrailingZeros(bits);
+      return (i4096 << 12) | ((i64 & 0x3F) << 6) | i1;
     }
+  }
 
-    @Override
-    public long cost() {
-      return cost;
+  @Override
+  public void or(DocIdSetIterator it) throws IOException {
+    for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
+      set(doc);
     }
+  }
 
+  @Override
+  public long ramBytesUsed() {
+    return ramBytesUsed;
   }
 
+  @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    return Collections.emptyList();
+  }
 }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/MockFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/MockFilter.java?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/MockFilter.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/MockFilter.java Fri Oct 24 07:32:19 2014
@@ -18,7 +18,7 @@ package org.apache.lucene.search;
  */
 
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.util.FixedBitDocIdSet;
+import org.apache.lucene.util.BitDocIdSet;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.Bits;
 
@@ -29,7 +29,7 @@ public class MockFilter extends Filter {
   public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) {
     wasCalled = true;
     FixedBitSet bits = new FixedBitSet(context.reader().maxDoc());
-    return new FixedBitDocIdSet(bits);
+    return new BitDocIdSet(bits);
   }
 
   public void clear() {

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/SingleDocTestFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/SingleDocTestFilter.java?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/SingleDocTestFilter.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/SingleDocTestFilter.java Fri Oct 24 07:32:19 2014
@@ -21,7 +21,7 @@ import java.io.IOException;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.FixedBitDocIdSet;
+import org.apache.lucene.util.BitDocIdSet;
 import org.apache.lucene.util.FixedBitSet;
 
 public class SingleDocTestFilter extends Filter {
@@ -36,6 +36,6 @@ public class SingleDocTestFilter extends
     FixedBitSet bits = new FixedBitSet(context.reader().maxDoc());
     bits.set(doc);
     if (acceptDocs != null && !acceptDocs.get(doc)) bits.clear(doc);
-    return new FixedBitDocIdSet(bits);
+    return new BitDocIdSet(bits);
   }
 }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestCachingWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestCachingWrapperFilter.java?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestCachingWrapperFilter.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestCachingWrapperFilter.java Fri Oct 24 07:32:19 2014
@@ -32,7 +32,7 @@ import org.apache.lucene.index.SlowCompo
 import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.FixedBitDocIdSet;
+import org.apache.lucene.util.BitDocIdSet;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
@@ -265,7 +265,7 @@ public class TestCachingWrapperFilter ex
     assertDocIdSetCacheable(reader, new Filter() {
       @Override
       public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) {
-        return new FixedBitDocIdSet(new FixedBitSet(context.reader().maxDoc()));
+        return new BitDocIdSet(new FixedBitSet(context.reader().maxDoc()));
       }
     }, true);
 

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilteredQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilteredQuery.java?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilteredQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilteredQuery.java Fri Oct 24 07:32:19 2014
@@ -34,7 +34,7 @@ import org.apache.lucene.search.BooleanC
 import org.apache.lucene.search.FilteredQuery.FilterStrategy;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.FixedBitDocIdSet;
+import org.apache.lucene.util.BitDocIdSet;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
@@ -105,7 +105,7 @@ public class TestFilteredQuery extends L
         FixedBitSet bitset = new FixedBitSet(context.reader().maxDoc());
         if (acceptDocs.get(1)) bitset.set(1);
         if (acceptDocs.get(3)) bitset.set(3);
-        return new FixedBitDocIdSet(bitset);
+        return new BitDocIdSet(bitset);
       }
     };
   }
@@ -186,7 +186,7 @@ public class TestFilteredQuery extends L
         assertNull("acceptDocs should be null, as we have an index without deletions", acceptDocs);
         FixedBitSet bitset = new FixedBitSet(context.reader().maxDoc());
         bitset.set(0, Math.min(5, bitset.length()));
-        return new FixedBitDocIdSet(bitset);
+        return new BitDocIdSet(bitset);
       }
     };
   }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilteredSearch.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilteredSearch.java?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilteredSearch.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilteredSearch.java Fri Oct 24 07:32:19 2014
@@ -31,7 +31,7 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.FixedBitDocIdSet;
+import org.apache.lucene.util.BitDocIdSet;
 import org.apache.lucene.util.FixedBitSet;
 
 
@@ -99,7 +99,7 @@ public class TestFilteredSearch extends 
           set.set(docId-docBase);
         }
       }
-      return set.cardinality() == 0 ? null : new FixedBitDocIdSet(set);
+      return set.cardinality() == 0 ? null : new BitDocIdSet(set);
     }
   }
 

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java Fri Oct 24 07:32:19 2014
@@ -14,7 +14,7 @@ import org.apache.lucene.index.LeafReade
 import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.FixedBitDocIdSet;
+import org.apache.lucene.util.BitDocIdSet;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.LuceneTestCase;
 
@@ -146,7 +146,7 @@ public class TestScorerPerf extends Luce
       @Override
       public DocIdSet getDocIdSet (LeafReaderContext context, Bits acceptDocs) {
         assertNull("acceptDocs should be null, as we have an index without deletions", acceptDocs);
-        return new FixedBitDocIdSet(rnd);
+        return new BitDocIdSet(rnd);
       }
     });
     bq.add(q, BooleanClause.Occur.MUST);

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestSortRandom.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestSortRandom.java?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestSortRandom.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestSortRandom.java Fri Oct 24 07:32:19 2014
@@ -39,7 +39,7 @@ import org.apache.lucene.search.BooleanC
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.FixedBitDocIdSet;
+import org.apache.lucene.util.BitDocIdSet;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
@@ -261,7 +261,7 @@ public class TestSortRandom extends Luce
         }
       }
 
-      return new FixedBitDocIdSet(bits);
+      return new BitDocIdSet(bits);
     }
   }
 }

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=1634012&r1=1634011&r2=1634012&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 Fri Oct 24 07:32:19 2014
@@ -45,6 +45,16 @@ public class TestDocIdSetBuilder extends
     }
   }
 
+  public void testFull() throws IOException {
+    final int maxDoc = 1 + random().nextInt(1000);
+    DocIdSetBuilder builder = new DocIdSetBuilder(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);
     DocIdSetBuilder builder = new DocIdSetBuilder(maxDoc);
@@ -60,8 +70,8 @@ public class TestDocIdSetBuilder extends
       builder.or(b.build().iterator());
     }
     DocIdSet result = builder.build();
-    assertTrue(result instanceof SparseFixedBitDocIdSet);
-    assertEquals(new FixedBitDocIdSet(ref), result);
+    assertTrue(result instanceof BitDocIdSet);
+    assertEquals(new BitDocIdSet(ref), result);
   }
 
   public void testDense() throws IOException {
@@ -84,8 +94,8 @@ public class TestDocIdSetBuilder extends
       builder.or(b.build().iterator());
     }
     DocIdSet result = builder.build();
-    assertTrue(result instanceof FixedBitDocIdSet);
-    assertEquals(new FixedBitDocIdSet(ref), result);
+    assertTrue(result instanceof BitDocIdSet);
+    assertEquals(new BitDocIdSet(ref), result);
   }
 
 }

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestFixedBitDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestFixedBitDocIdSet.java?rev=1634012&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestFixedBitDocIdSet.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestFixedBitDocIdSet.java Fri Oct 24 07:32:19 2014
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+package org.apache.lucene.util;
+
+import java.io.IOException;
+import java.util.BitSet;
+
+public class TestFixedBitDocIdSet extends BaseDocIdSetTestCase<BitDocIdSet> {
+
+  @Override
+  public BitDocIdSet copyOf(BitSet bs, int length) throws IOException {
+    final FixedBitSet set = new FixedBitSet(length);
+    for (int doc = bs.nextSetBit(0); doc != -1; doc = bs.nextSetBit(doc + 1)) {
+      set.set(doc);
+    }
+    return new BitDocIdSet(set);
+  }
+  
+}

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestFixedBitSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestFixedBitSet.java?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestFixedBitSet.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestFixedBitSet.java Fri Oct 24 07:32:19 2014
@@ -1,3 +1,5 @@
+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
@@ -15,26 +17,23 @@
  * limitations under the License.
  */
 
-package org.apache.lucene.util;
-
 import java.io.IOException;
-import java.util.BitSet;
 
 import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.util.FixedBitSet.FixedBitSetIterator;
 
-public class TestFixedBitSet extends BaseDocIdSetTestCase<FixedBitDocIdSet> {
+public class TestFixedBitSet extends BaseBitSetTestCase<FixedBitSet> {
 
   @Override
-  public FixedBitDocIdSet copyOf(BitSet bs, int length) throws IOException {
+  public FixedBitSet copyOf(BitSet bs, int length) throws IOException {
     final FixedBitSet set = new FixedBitSet(length);
-    for (int doc = bs.nextSetBit(0); doc != -1; doc = bs.nextSetBit(doc + 1)) {
+    for (int doc = bs.nextSetBit(0); doc != DocIdSetIterator.NO_MORE_DOCS; doc = doc + 1 >= length ? DocIdSetIterator.NO_MORE_DOCS : bs.nextSetBit(doc + 1)) {
       set.set(doc);
     }
-    return new FixedBitDocIdSet(set);
+    return set;
   }
 
-  void doGet(BitSet a, FixedBitSet b) {
+
+  void doGet(java.util.BitSet a, FixedBitSet b) {
     int max = b.length();
     for (int i=0; i<max; i++) {
       if (a.get(i) != b.get(i)) {
@@ -43,16 +42,19 @@ public class TestFixedBitSet extends Bas
     }
   }
 
-  void doNextSetBit(BitSet a, FixedBitSet b) {
+  void doNextSetBit(java.util.BitSet a, FixedBitSet b) {
     int aa=-1,bb=-1;
     do {
       aa = a.nextSetBit(aa+1);
-      bb = bb < b.length()-1 ? b.nextSetBit(bb+1) : -1;
+      if (aa == -1) {
+        aa = DocIdSetIterator.NO_MORE_DOCS;
+      }
+      bb = bb < b.length()-1 ? b.nextSetBit(bb+1) : DocIdSetIterator.NO_MORE_DOCS;
       assertEquals(aa,bb);
-    } while (aa>=0);
+    } while (aa != DocIdSetIterator.NO_MORE_DOCS);
   }
 
-  void doPrevSetBit(BitSet a, FixedBitSet b) {
+  void doPrevSetBit(java.util.BitSet a, FixedBitSet b) {
     int aa = a.size() + random().nextInt(100);
     int bb = aa;
     do {
@@ -75,14 +77,14 @@ public class TestFixedBitSet extends Bas
   }
 
   // test interleaving different FixedBitSetIterator.next()/skipTo()
-  void doIterate(BitSet a, FixedBitSet b, int mode) throws IOException {
+  void doIterate(java.util.BitSet a, FixedBitSet b, int mode) throws IOException {
     if (mode==1) doIterate1(a, b);
     if (mode==2) doIterate2(a, b);
   }
 
-  void doIterate1(BitSet a, FixedBitSet b) throws IOException {
+  void doIterate1(java.util.BitSet a, FixedBitSet b) throws IOException {
     int aa=-1,bb=-1;
-    DocIdSetIterator iterator = new FixedBitSetIterator(b, 0);
+    DocIdSetIterator iterator = new BitSetIterator(b, 0);
     do {
       aa = a.nextSetBit(aa+1);
       bb = (bb < b.length() && random().nextBoolean()) ? iterator.nextDoc() : iterator.advance(bb + 1);
@@ -90,9 +92,9 @@ public class TestFixedBitSet extends Bas
     } while (aa>=0);
   }
 
-  void doIterate2(BitSet a, FixedBitSet b) throws IOException {
+  void doIterate2(java.util.BitSet a, FixedBitSet b) throws IOException {
     int aa=-1,bb=-1;
-    DocIdSetIterator iterator = new FixedBitSetIterator(b, 0);
+    DocIdSetIterator iterator = new BitSetIterator(b, 0);
     do {
       aa = a.nextSetBit(aa+1);
       bb = random().nextBoolean() ? iterator.nextDoc() : iterator.advance(bb + 1);
@@ -101,12 +103,12 @@ public class TestFixedBitSet extends Bas
   }
 
   void doRandomSets(int maxSize, int iter, int mode) throws IOException {
-    BitSet a0=null;
+    java.util.BitSet a0=null;
     FixedBitSet b0=null;
 
     for (int i=0; i<iter; i++) {
       int sz = TestUtil.nextInt(random(), 2, maxSize);
-      BitSet a = new BitSet(sz);
+      java.util.BitSet a = new java.util.BitSet(sz);
       FixedBitSet b = new FixedBitSet(sz);
 
       // test the various ways of setting bits
@@ -148,14 +150,14 @@ public class TestFixedBitSet extends Bas
       int fromIndex, toIndex;
       fromIndex = random().nextInt(sz/2);
       toIndex = fromIndex + random().nextInt(sz - fromIndex);
-      BitSet aa = (BitSet)a.clone(); aa.flip(fromIndex,toIndex);
+      java.util.BitSet aa = (java.util.BitSet)a.clone(); aa.flip(fromIndex,toIndex);
       FixedBitSet bb = b.clone(); bb.flip(fromIndex,toIndex);
 
       doIterate(aa,bb, mode);   // a problem here is from flip or doIterate
 
       fromIndex = random().nextInt(sz/2);
       toIndex = fromIndex + random().nextInt(sz - fromIndex);
-      aa = (BitSet)a.clone(); aa.clear(fromIndex,toIndex);
+      aa = (java.util.BitSet)a.clone(); aa.clear(fromIndex,toIndex);
       bb = b.clone(); bb.clear(fromIndex,toIndex);
 
       doNextSetBit(aa,bb); // a problem here is from clear() or nextSetBit
@@ -164,7 +166,7 @@ public class TestFixedBitSet extends Bas
 
       fromIndex = random().nextInt(sz/2);
       toIndex = fromIndex + random().nextInt(sz - fromIndex);
-      aa = (BitSet)a.clone(); aa.set(fromIndex,toIndex);
+      aa = (java.util.BitSet)a.clone(); aa.set(fromIndex,toIndex);
       bb = b.clone(); bb.set(fromIndex,toIndex);
 
       doNextSetBit(aa,bb); // a problem here is from set() or nextSetBit
@@ -174,10 +176,10 @@ public class TestFixedBitSet extends Bas
       if (b0 != null && b0.length() <= b.length()) {
         assertEquals(a.cardinality(), b.cardinality());
 
-        BitSet a_and = (BitSet)a.clone(); a_and.and(a0);
-        BitSet a_or = (BitSet)a.clone(); a_or.or(a0);
-        BitSet a_xor = (BitSet)a.clone(); a_xor.xor(a0);
-        BitSet a_andn = (BitSet)a.clone(); a_andn.andNot(a0);
+        java.util.BitSet a_and = (java.util.BitSet)a.clone(); a_and.and(a0);
+        java.util.BitSet a_or = (java.util.BitSet)a.clone(); a_or.or(a0);
+        java.util.BitSet a_xor = (java.util.BitSet)a.clone(); a_xor.xor(a0);
+        java.util.BitSet a_andn = (java.util.BitSet)a.clone(); a_andn.andNot(a0);
 
         FixedBitSet b_and = b.clone(); assertEquals(b,b_and); b_and.and(b0);
         FixedBitSet b_or = b.clone(); b_or.or(b0);
@@ -297,8 +299,8 @@ public class TestFixedBitSet extends Bas
     return bs;
   }
 
-  private BitSet makeBitSet(int[] a) {
-    BitSet bs = new BitSet();
+  private java.util.BitSet makeBitSet(int[] a) {
+    java.util.BitSet bs = new java.util.BitSet();
     for (int e: a) {
       bs.set(e);
     }
@@ -307,7 +309,7 @@ public class TestFixedBitSet extends Bas
 
   private void checkPrevSetBitArray(int [] a, int numBits) {
     FixedBitSet obs = makeFixedBitSet(a, numBits);
-    BitSet bs = makeBitSet(a);
+    java.util.BitSet bs = makeBitSet(a);
     doPrevSetBit(bs, obs);
   }
 
@@ -320,7 +322,7 @@ public class TestFixedBitSet extends Bas
   
   private void checkNextSetBitArray(int [] a, int numBits) {
     FixedBitSet obs = makeFixedBitSet(a, numBits);
-    BitSet bs = makeBitSet(a);
+    java.util.BitSet bs = makeBitSet(a);
     doNextSetBit(bs, obs);
   }
   
@@ -360,5 +362,4 @@ public class TestFixedBitSet extends Bas
     assertTrue(bits.get(1));
     assertFalse(newBits.get(1));
   }
-  
 }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestNotDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestNotDocIdSet.java?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestNotDocIdSet.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestNotDocIdSet.java Fri Oct 24 07:32:19 2014
@@ -30,7 +30,7 @@ public class TestNotDocIdSet extends Bas
     for (int doc = bs.nextClearBit(0); doc < length; doc = bs.nextClearBit(doc + 1)) {
       set.set(doc);
     }
-    return new NotDocIdSet(length, new FixedBitDocIdSet(set));
+    return new NotDocIdSet(length, new BitDocIdSet(set));
   }
 
   @Override
@@ -48,7 +48,7 @@ public class TestNotDocIdSet extends Bas
 
   public void testBits() throws IOException {
     assertNull(new NotDocIdSet(3, DocIdSet.EMPTY).bits());
-    assertNotNull(new NotDocIdSet(3, new FixedBitDocIdSet(new FixedBitSet(3))).bits());
+    assertNotNull(new NotDocIdSet(3, new BitDocIdSet(new FixedBitSet(3))).bits());
   }
 
 }

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestSparseFixedBitDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestSparseFixedBitDocIdSet.java?rev=1634012&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestSparseFixedBitDocIdSet.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestSparseFixedBitDocIdSet.java Fri Oct 24 07:32:19 2014
@@ -0,0 +1,60 @@
+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.ArrayList;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
+
+public class TestSparseFixedBitDocIdSet extends BaseDocIdSetTestCase<BitDocIdSet> {
+
+  @Override
+  public BitDocIdSet copyOf(BitSet bs, int length) throws IOException {
+    final SparseFixedBitSet set = new SparseFixedBitSet(length);
+    // SparseFixedBitSet can be sensitive to the order of insertion so
+    // randomize insertion a bit
+    List<Integer> buffer = new ArrayList<>();
+    for (int doc = bs.nextSetBit(0); doc != -1; doc = bs.nextSetBit(doc + 1)) {
+      buffer.add(doc);
+      if (buffer.size() >= 100000) {
+        Collections.shuffle(buffer, random());
+        for (int i : buffer) {
+          set.set(i);
+        }
+        buffer.clear();
+      }
+    }
+    Collections.shuffle(buffer, random());
+    for (int i : buffer) {
+      set.set(i);
+    }
+    return new BitDocIdSet(set, set.approximateCardinality());
+  }
+
+  @Override
+  public void assertEquals(int numBits, BitSet ds1, BitDocIdSet ds2) throws IOException {
+    for (int i = 0; i < numBits; ++i) {
+      assertEquals(ds1.get(i), ds2.bits().get(i));
+    }
+    assertEquals(ds1.cardinality(), ds2.bits().cardinality());
+    super.assertEquals(numBits, ds1, ds2);
+  }
+
+}

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestSparseFixedBitSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestSparseFixedBitSet.java?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestSparseFixedBitSet.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestSparseFixedBitSet.java Fri Oct 24 07:32:19 2014
@@ -18,43 +18,18 @@ package org.apache.lucene.util;
  */
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.BitSet;
-import java.util.Collections;
-import java.util.List;
 
-public class TestSparseFixedBitSet extends BaseDocIdSetTestCase<SparseFixedBitDocIdSet> {
+import org.apache.lucene.search.DocIdSetIterator;
 
-  @Override
-  public SparseFixedBitDocIdSet copyOf(BitSet bs, int length) throws IOException {
-    final SparseFixedBitSet set = new SparseFixedBitSet(length);
-    // SparseFixedBitSet can be sensitive to the order of insertion so
-    // randomize insertion a bit
-    List<Integer> buffer = new ArrayList<>();
-    for (int doc = bs.nextSetBit(0); doc != -1; doc = bs.nextSetBit(doc + 1)) {
-      buffer.add(doc);
-      if (buffer.size() >= 100000) {
-        Collections.shuffle(buffer, random());
-        for (int i : buffer) {
-          set.set(i);
-        }
-        buffer.clear();
-      }
-    }
-    Collections.shuffle(buffer, random());
-    for (int i : buffer) {
-      set.set(i);
-    }
-    return new SparseFixedBitDocIdSet(set, set.approximateCardinality());
-  }
+public class TestSparseFixedBitSet extends BaseBitSetTestCase<SparseFixedBitSet> {
 
   @Override
-  public void assertEquals(int numBits, BitSet ds1, SparseFixedBitDocIdSet ds2) throws IOException {
-    for (int i = 0; i < numBits; ++i) {
-      assertEquals(ds1.get(i), ds2.bits().get(i));
+  public SparseFixedBitSet copyOf(BitSet bs, int length) throws IOException {
+    final SparseFixedBitSet set = new SparseFixedBitSet(length);
+    for (int doc = bs.nextSetBit(0); doc != DocIdSetIterator.NO_MORE_DOCS; doc = doc + 1 >= length ? DocIdSetIterator.NO_MORE_DOCS : bs.nextSetBit(doc + 1)) {
+      set.set(doc);
     }
-    assertEquals(ds1.cardinality(), ds2.bits().cardinality());
-    super.assertEquals(numBits, ds1, ds2);
+    return set;
   }
 
   public void testApproximateCardinality() {
@@ -70,12 +45,11 @@ public class TestSparseFixedBitSet exten
   public void testApproximateCardinalityOnDenseSet() {
     // this tests that things work as expected in approximateCardinality when
     // all longs are different than 0, in which case we divide by zero
-    final int numDocs = 70;//TestUtil.nextInt(random(), 1, 10000);
+    final int numDocs = TestUtil.nextInt(random(), 1, 10000);
     final SparseFixedBitSet set = new SparseFixedBitSet(numDocs);
     for (int i = 0; i < set.length(); ++i) {
       set.set(i);
     }
     assertEquals(numDocs, set.approximateCardinality());
   }
-
 }

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java Fri Oct 24 07:32:19 2014
@@ -330,7 +330,7 @@ class DrillSidewaysScorer extends BulkSc
       // Fold in baseScorer, using advance:
       int filledCount = 0;
       int slot0 = 0;
-      while (slot0 < CHUNK && (slot0 = seen.nextSetBit(slot0)) != -1) {
+      while (slot0 < CHUNK && (slot0 = seen.nextSetBit(slot0)) != DocIdSetIterator.NO_MORE_DOCS) {
         int ddDocID = docIDs[slot0];
         assert ddDocID != -1;
 

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java Fri Oct 24 07:32:19 2014
@@ -40,7 +40,7 @@ import org.apache.lucene.search.TopField
 import org.apache.lucene.search.TopFieldDocs;
 import org.apache.lucene.search.TopScoreDocCollector;
 import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.FixedBitDocIdSet;
+import org.apache.lucene.util.BitDocIdSet;
 import org.apache.lucene.util.FixedBitSet;
 
 /** Collects hits for subsequent faceting.  Once you've run
@@ -129,7 +129,7 @@ public class FacetsCollector extends Sim
       
       @Override
       public DocIdSet getDocIdSet() {
-        return new FixedBitDocIdSet(bits);
+        return new BitDocIdSet(bits);
       }
     };
   }

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/RandomSamplingFacetsCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/RandomSamplingFacetsCollector.java?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/RandomSamplingFacetsCollector.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/RandomSamplingFacetsCollector.java Fri Oct 24 07:32:19 2014
@@ -26,7 +26,7 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.util.FixedBitDocIdSet;
+import org.apache.lucene.util.BitDocIdSet;
 import org.apache.lucene.util.FixedBitSet;
 
 /**
@@ -213,7 +213,7 @@ public class RandomSamplingFacetsCollect
         }
       }
       
-      return new MatchingDocs(docs.context, new FixedBitDocIdSet(sampleDocs), docs.totalHits, null);
+      return new MatchingDocs(docs.context, new BitDocIdSet(sampleDocs), docs.totalHits, null);
     } catch (IOException e) {
       throw new RuntimeException();
     }

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java Fri Oct 24 07:32:19 2014
@@ -58,7 +58,7 @@ import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.FixedBitDocIdSet;
+import org.apache.lucene.util.BitDocIdSet;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.InPlaceMergeSorter;
@@ -658,7 +658,7 @@ public class TestDrillSideways extends F
                   bits.set(docID);
                 }
               }
-              return new FixedBitDocIdSet(bits);
+              return new BitDocIdSet(bits);
             }
           };
       } else {

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java Fri Oct 24 07:32:19 2014
@@ -65,7 +65,7 @@ import org.apache.lucene.search.NumericR
 import org.apache.lucene.search.NumericRangeQuery;
 import org.apache.lucene.search.QueryWrapperFilter;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.FixedBitDocIdSet;
+import org.apache.lucene.util.BitDocIdSet;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.TestUtil;
@@ -927,7 +927,7 @@ public class TestRangeFacetCounts extend
             final FixedBitSet cached = new FixedBitSet(reader.maxDoc());
             filterWasUsed.set(true);
             cached.or(iterator);
-            return new FixedBitDocIdSet(cached);
+            return new BitDocIdSet(cached);
           }
         };
     } else {

Modified: lucene/dev/trunk/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java (original)
+++ lucene/dev/trunk/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java Fri Oct 24 07:32:19 2014
@@ -43,6 +43,7 @@ import org.apache.lucene.index.RandomInd
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.BytesRefFieldSource;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.Sort;
@@ -405,7 +406,7 @@ public class AllGroupHeadsCollectorTest 
       expected.set(expectedDoc);
     }
 
-    for (int docId = expected.nextSetBit(0); docId != -1; docId = docId + 1 >= expected.length() ? -1 : expected.nextSetBit(docId + 1)) {
+    for (int docId = expected.nextSetBit(0); docId != DocIdSetIterator.NO_MORE_DOCS; docId = docId + 1 >= expected.length() ? DocIdSetIterator.NO_MORE_DOCS : expected.nextSetBit(docId + 1)) {
       if (!actual.get(docId)) {
         return false;
       }

Modified: lucene/dev/trunk/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterPhraseTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterPhraseTest.java?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterPhraseTest.java (original)
+++ lucene/dev/trunk/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterPhraseTest.java Fri Oct 24 07:32:19 2014
@@ -35,6 +35,7 @@ import org.apache.lucene.index.Directory
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.PhraseQuery;
 import org.apache.lucene.search.Query;
@@ -143,7 +144,7 @@ public class HighlighterPhraseTest exten
       final Highlighter highlighter = new Highlighter(
           new SimpleHTMLFormatter(), new SimpleHTMLEncoder(),
           new QueryScorer(phraseQuery));
-      for (int position = bitset.nextSetBit(0); position >= 0 && position < maxDoc-1; position = bitset
+      for (int position = bitset.nextSetBit(0); position < maxDoc-1; position = bitset
           .nextSetBit(position + 1)) {
         assertEquals(0, position);
         final TokenStream tokenStream = TokenSources.getTokenStream(

Modified: lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/FixedBitSetCachingWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/FixedBitSetCachingWrapperFilter.java?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/FixedBitSetCachingWrapperFilter.java (original)
+++ lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/FixedBitSetCachingWrapperFilter.java Fri Oct 24 07:32:19 2014
@@ -26,7 +26,7 @@ import org.apache.lucene.search.CachingW
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.Filter;
-import org.apache.lucene.util.FixedBitDocIdSet;
+import org.apache.lucene.util.BitDocIdSet;
 import org.apache.lucene.util.FixedBitSet;
 
 /** A {@link CachingWrapperFilter} that caches sets using a {@link FixedBitSet},
@@ -43,7 +43,7 @@ public final class FixedBitSetCachingWra
       throws IOException {
     if (docIdSet == null) {
       return EMPTY;
-    } else if (docIdSet instanceof FixedBitDocIdSet) {
+    } else if (docIdSet instanceof BitDocIdSet) {
       // this is different from CachingWrapperFilter: even when the DocIdSet is
       // cacheable, we convert it to a FixedBitSet since we require all the
       // cached filters to be FixedBitSets
@@ -55,7 +55,7 @@ public final class FixedBitSetCachingWra
       } else {
         final FixedBitSet copy = new FixedBitSet(reader.maxDoc());
         copy.or(it);
-        return new FixedBitDocIdSet(copy);
+        return new BitDocIdSet(copy);
       }
     }
   }

Modified: lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java (original)
+++ lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java Fri Oct 24 07:32:19 2014
@@ -36,11 +36,11 @@ import org.apache.lucene.search.LeafColl
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.BitSetIterator;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.FixedBitSet;
-import org.apache.lucene.util.FixedBitSet.FixedBitSetIterator;
 
 class TermsIncludingScoreQuery extends Query {
 
@@ -329,7 +329,7 @@ class TermsIncludingScoreQuery extends Q
       FixedBitSet matchingDocs = new FixedBitSet(maxDoc);
       this.scores = new float[maxDoc];
       fillDocsAndScores(matchingDocs, acceptDocs, termsEnum);
-      this.matchingDocsIterator = new FixedBitSetIterator(matchingDocs, cost);
+      this.matchingDocsIterator = new BitSetIterator(matchingDocs, cost);
       this.cost = cost;
     }
 

Modified: lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java?rev=1634012&r1=1634011&r2=1634012&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java (original)
+++ lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java Fri Oct 24 07:32:19 2014
@@ -33,7 +33,7 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.FixedBitDocIdSet;
+import org.apache.lucene.util.BitDocIdSet;
 import org.apache.lucene.util.FixedBitSet;
 
 /**
@@ -145,11 +145,11 @@ public class ToChildBlockJoinQuery exten
         // No matches
         return null;
       }
-      if (!(parents instanceof FixedBitDocIdSet)) {
-        throw new IllegalStateException("parentFilter must return FixedBitSet; got " + parents);
+      if (!(parents.bits() instanceof FixedBitSet)) {
+        throw new IllegalStateException("parentFilter must return FixedBitSet; got " + parents.bits());
       }
 
-      return new ToChildBlockJoinScorer(this, parentScorer, ((FixedBitDocIdSet) parents).bits(), doScores, acceptDocs);
+      return new ToChildBlockJoinScorer(this, parentScorer, (FixedBitSet) parents.bits(), doScores, acceptDocs);
     }
 
     @Override