You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2013/01/14 13:07:43 UTC

svn commit: r1432894 [1/2] - in /lucene/dev/branches/branch_4x: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/util/ lucene/facet/ lucene/facet/src/java/org/apache/lucene/facet/associations/ lucene/facet/src/java/org/apache/lucene/facet...

Author: shaie
Date: Mon Jan 14 12:07:42 2013
New Revision: 1432894

URL: http://svn.apache.org/viewvc?rev=1432894&view=rev
Log:
LUCENE-4683: Change Aggregator and CategoryListIterator to be per-segment

Added:
    lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/TestStandardFacetsAccumulator.java
      - copied unchanged from r1432890, lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestStandardFacetsAccumulator.java
    lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/util/AssertingCategoryListIterator.java
      - copied unchanged from r1432890, lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/util/AssertingCategoryListIterator.java
Modified:
    lucene/dev/branches/branch_4x/   (props changed)
    lucene/dev/branches/branch_4x/lucene/   (props changed)
    lucene/dev/branches/branch_4x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_4x/lucene/core/   (props changed)
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java
    lucene/dev/branches/branch_4x/lucene/facet/   (props changed)
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsPayloadIterator.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/FloatAssociationsPayloadIterator.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/IntAssociationsPayloadIterator.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/index/params/CategoryListParams.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/AdaptiveFacetsAccumulator.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/CategoryListIterator.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/PayloadCategoryListIteraor.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/PayloadIterator.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/ScoredDocIdCollector.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/StandardFacetsAccumulator.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/TotalFacetCounts.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/Aggregator.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/CountingAggregator.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/ScoringAggregator.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationFloatSumAggregator.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationIntSumAggregator.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/cache/CategoryListData.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/params/CountFacetRequest.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/params/FacetRequest.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/params/ScoreFacetRequest.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationFloatSumFacetRequest.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationIntSumFacetRequest.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/sampling/Sampler.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/sampling/TakmiSampleFixer.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/util/MultiCategoryListIterator.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/util/ScoredDocIdsUtils.java
    lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/FacetTestBase.java
    lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/AdaptiveAccumulatorTest.java
    lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/CategoryListIteratorTest.java
    lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/TestCategoryListCache.java
    lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/params/MultiCategoryListIteratorTest.java
    lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/sampling/BaseSampleTestTopK.java
    lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/util/TestScoredDocIDsUtils.java

Modified: lucene/dev/branches/branch_4x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/CHANGES.txt?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_4x/lucene/CHANGES.txt Mon Jan 14 12:07:42 2013
@@ -100,6 +100,10 @@ Changes in backwards compatibility polic
   result, few other classes such as Aggregator and CategoryListIterator were
   changed to handle bulk category ordinals. (Shai Erera)
 
+* LUCENE-4683: CategoryListIterator and Aggregator are now per-segment. As such
+  their implementations no longer take a top-level IndexReader in the constructor
+  but rather implement a setNextReader. (Shai Erera)
+  
 New Features
 
 * LUCENE-4226: New experimental StoredFieldsFormat that compresses chunks of

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java Mon Jan 14 12:07:42 2013
@@ -38,7 +38,7 @@ import org.apache.lucene.search.DocIdSet
 
 public final class FixedBitSet extends DocIdSet implements Bits {
   private final long[] bits;
-  private int numBits;
+  private final int numBits;
 
   /** returns the number of 64 bit words it would take to hold numBits */
   public static int bits2words(int numBits) {

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsPayloadIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsPayloadIterator.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsPayloadIterator.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsPayloadIterator.java Mon Jan 14 12:07:42 2013
@@ -3,7 +3,7 @@ package org.apache.lucene.facet.associat
 import java.io.IOException;
 
 import org.apache.lucene.facet.search.PayloadIterator;
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.util.BytesRef;
@@ -46,13 +46,22 @@ public abstract class AssociationsPayloa
    * It is assumed that all association values can be deserialized with the
    * given {@link CategoryAssociation}.
    */
-  public AssociationsPayloadIterator(IndexReader reader, String field, T association) throws IOException {
-    pi = new PayloadIterator(reader, new Term(field, association.getCategoryListID()));
-    hasAssociations = pi.init();
+  public AssociationsPayloadIterator(String field, T association) throws IOException {
+    pi = new PayloadIterator(new Term(field, association.getCategoryListID()));
     this.association = association;
   }
 
   /**
+   * Sets the {@link AtomicReaderContext} for which {@link #setNextDoc(int)}
+   * calls will be made. Returns true iff this reader has associations for any
+   * of the documents belonging to the association given to the constructor.
+   */
+  public final boolean setNextReader(AtomicReaderContext context) throws IOException {
+    hasAssociations = pi.setNextReader(context);
+    return hasAssociations;
+  }
+  
+  /**
    * Skip to the requested document. Returns true iff the document has category
    * association values and they were read successfully. Associations are
    * handled through {@link #handleAssociation(int, CategoryAssociation)} by

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/FloatAssociationsPayloadIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/FloatAssociationsPayloadIterator.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/FloatAssociationsPayloadIterator.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/FloatAssociationsPayloadIterator.java Mon Jan 14 12:07:42 2013
@@ -2,7 +2,6 @@ package org.apache.lucene.facet.associat
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.util.collections.IntToFloatMap;
 
 /*
@@ -31,9 +30,8 @@ public class FloatAssociationsPayloadIte
 
   private final IntToFloatMap ordinalAssociations = new IntToFloatMap();
 
-  public FloatAssociationsPayloadIterator(IndexReader reader, String field, CategoryFloatAssociation association) 
-      throws IOException {
-    super(reader, field, association);
+  public FloatAssociationsPayloadIterator(String field, CategoryFloatAssociation association) throws IOException {
+    super(field, association);
   }
 
   @Override

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/IntAssociationsPayloadIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/IntAssociationsPayloadIterator.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/IntAssociationsPayloadIterator.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/IntAssociationsPayloadIterator.java Mon Jan 14 12:07:42 2013
@@ -2,7 +2,6 @@ package org.apache.lucene.facet.associat
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.util.collections.IntToIntMap;
 
 /*
@@ -31,9 +30,8 @@ public class IntAssociationsPayloadItera
 
   private final IntToIntMap ordinalAssociations = new IntToIntMap();
 
-  public IntAssociationsPayloadIterator(IndexReader reader, String field, CategoryIntAssociation association) 
-      throws IOException {
-    super(reader, field, association);
+  public IntAssociationsPayloadIterator(String field, CategoryIntAssociation association) throws IOException {
+    super(field, association);
   }
 
   @Override

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/index/params/CategoryListParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/index/params/CategoryListParams.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/index/params/CategoryListParams.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/index/params/CategoryListParams.java Mon Jan 14 12:07:42 2013
@@ -3,13 +3,10 @@ package org.apache.lucene.facet.index.pa
 import java.io.IOException;
 import java.io.Serializable;
 
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.Term;
-
 import org.apache.lucene.facet.search.CategoryListIterator;
 import org.apache.lucene.facet.search.PayloadCategoryListIteraor;
-import org.apache.lucene.facet.search.TotalFacetCounts;
 import org.apache.lucene.facet.util.PartitionsUtils;
+import org.apache.lucene.index.Term;
 import org.apache.lucene.util.encoding.DGapIntEncoder;
 import org.apache.lucene.util.encoding.IntDecoder;
 import org.apache.lucene.util.encoding.IntEncoder;
@@ -98,11 +95,6 @@ public class CategoryListParams implemen
     return new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new VInt8IntEncoder())));
   }
 
-  /**
-   * Equality is defined by the 'term' that defines this category list.  
-   * Sub-classes should override this method if a more complex calculation
-   * is needed to ensure equality. 
-   */
   @Override
   public boolean equals(Object o) {
     if (o == this) {
@@ -121,29 +113,16 @@ public class CategoryListParams implemen
     return this.term.equals(other.term);
   }
 
-  /**
-   * Hashcode is similar to {@link #equals(Object)}, in that it uses
-   * the term that defines this category list to derive the hashcode.
-   * Subclasses need to ensure that equality/hashcode is correctly defined,
-   * or there could be side-effects in the {@link TotalFacetCounts} caching 
-   * mechanism (as the filename for a Total Facet Counts array cache 
-   * is dependent on the hashCode, so it should consistently return the same
-   * hash for identity).
-   */
   @Override
   public int hashCode() {
     return this.hashCode;
   }
 
-  /**
-   * Create the category list iterator for the specified partition.
-   */
-  public CategoryListIterator createCategoryListIterator(IndexReader reader,
-      int partition) throws IOException {
+  /** Create the {@link CategoryListIterator} for the specified partition. */
+  public CategoryListIterator createCategoryListIterator(int partition) throws IOException {
     String categoryListTermStr = PartitionsUtils.partitionName(this, partition);
     Term payloadTerm = new Term(term.field(), categoryListTermStr);
-    return new PayloadCategoryListIteraor(reader, payloadTerm,
-        createEncoder().createMatchingDecoder());
+    return new PayloadCategoryListIteraor(payloadTerm, createEncoder().createMatchingDecoder());
   }
   
 }
\ No newline at end of file

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/AdaptiveFacetsAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/AdaptiveFacetsAccumulator.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/AdaptiveFacetsAccumulator.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/AdaptiveFacetsAccumulator.java Mon Jan 14 12:07:42 2013
@@ -50,7 +50,7 @@ public final class AdaptiveFacetsAccumul
    * Create an {@link AdaptiveFacetsAccumulator} 
    * @see StandardFacetsAccumulator#StandardFacetsAccumulator(FacetSearchParams, IndexReader, TaxonomyReader)
    */
-  public AdaptiveFacetsAccumulator(FacetSearchParams searchParams, IndexReader indexReader,
+  public AdaptiveFacetsAccumulator(FacetSearchParams searchParams, IndexReader indexReader, 
       TaxonomyReader taxonomyReader) {
     super(searchParams, indexReader, taxonomyReader);
   }

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/CategoryListIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/CategoryListIterator.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/CategoryListIterator.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/CategoryListIterator.java Mon Jan 14 12:07:42 2013
@@ -2,6 +2,7 @@ package org.apache.lucene.facet.search;
 
 import java.io.IOException;
 
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.util.IntsRef;
 
 /*
@@ -23,6 +24,8 @@ import org.apache.lucene.util.IntsRef;
 
 /**
  * An interface for obtaining the category ordinals of documents.
+ * {@link #getOrdinals(int, IntsRef)} calls are done with document IDs that are
+ * local to the reader given to {@link #setNextReader(AtomicReaderContext)}.
  * <p>
  * <b>NOTE:</b> this class operates as a key to a map, and therefore you should
  * implement {@code equals()} and {@code hashCode()} for proper behavior.
@@ -32,19 +35,20 @@ import org.apache.lucene.util.IntsRef;
 public interface CategoryListIterator {
 
   /**
-   * Initializes the iterator. This method must be called before any calls to
-   * {@link #getOrdinals(int, IntsRef)}, and its return value indicates whether there are
-   * any relevant documents for this iterator.
+   * Sets the {@link AtomicReaderContext} for which
+   * {@link #getOrdinals(int, IntsRef)} calls will be made. Returns true iff any
+   * of the documents in this reader have category ordinals. This method must be
+   * called before any calls to {@link #getOrdinals(int, IntsRef)}.
    */
-  public boolean init() throws IOException;
-
+  public boolean setNextReader(AtomicReaderContext context) throws IOException;
+  
   /**
    * Stores the category ordinals of the given document ID in the given
    * {@link IntsRef}, starting at position 0 upto {@link IntsRef#length}. Grows
    * the {@link IntsRef} if it is not large enough.
    * 
    * <p>
-   * <b>NOTE:</b> if the requested document does not category ordinals
+   * <b>NOTE:</b> if the requested document does not have category ordinals
    * associated with it, {@link IntsRef#length} is set to zero.
    */
   public void getOrdinals(int docID, IntsRef ints) throws IOException;

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/PayloadCategoryListIteraor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/PayloadCategoryListIteraor.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/PayloadCategoryListIteraor.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/PayloadCategoryListIteraor.java Mon Jan 14 12:07:42 2013
@@ -2,7 +2,7 @@ package org.apache.lucene.facet.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IntsRef;
@@ -34,17 +34,15 @@ import org.apache.lucene.util.encoding.I
 public class PayloadCategoryListIteraor implements CategoryListIterator {
 
   private final IntDecoder decoder;
-  private final IndexReader indexReader;
   private final Term term;
   private final PayloadIterator pi;
   private final int hashCode;
   
-  public PayloadCategoryListIteraor(IndexReader indexReader, Term term, IntDecoder decoder) throws IOException {
-    pi = new PayloadIterator(indexReader, term);
+  public PayloadCategoryListIteraor(Term term, IntDecoder decoder) throws IOException {
+    pi = new PayloadIterator(term);
     this.decoder = decoder;
-    hashCode = indexReader.hashCode() ^ term.hashCode();
+    hashCode = term.hashCode();
     this.term = term;
-    this.indexReader = indexReader;
   }
 
   @Override
@@ -58,7 +56,7 @@ public class PayloadCategoryListIteraor 
     }
     
     // Hash codes are the same, check equals() to avoid cases of hash-collisions.
-    return indexReader.equals(that.indexReader) && term.equals(that.term);
+    return term.equals(that.term);
   }
 
   @Override
@@ -67,8 +65,8 @@ public class PayloadCategoryListIteraor 
   }
 
   @Override
-  public boolean init() throws IOException {
-    return pi.init();
+  public boolean setNextReader(AtomicReaderContext context) throws IOException {
+    return pi.setNextReader(context);
   }
   
   @Override

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/PayloadIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/PayloadIterator.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/PayloadIterator.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/PayloadIterator.java Mon Jan 14 12:07:42 2013
@@ -1,12 +1,10 @@
 package org.apache.lucene.facet.search;
 
 import java.io.IOException;
-import java.util.Iterator;
 
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.Fields;
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
@@ -42,99 +40,75 @@ import org.apache.lucene.util.BytesRef;
  */
 public class PayloadIterator {
 
-  protected BytesRef data;
-
   private TermsEnum reuseTE;
-  private DocsAndPositionsEnum currentDPE;
+  private DocsAndPositionsEnum dpe;
   private boolean hasMore;
-  private int curDocID, curDocBase;
+  private int curDocID;
   
-  private final Iterator<AtomicReaderContext> leaves;
   private final Term term;
 
-  public PayloadIterator(IndexReader indexReader, Term term) throws IOException {
-    leaves = indexReader.leaves().iterator();
+  public PayloadIterator(Term term) throws IOException {
     this.term = term;
   }
 
-  private void nextSegment() throws IOException {
+  /**
+   * Sets the {@link AtomicReaderContext} for which {@link #getPayload(int)}
+   * calls will be made. Returns true iff this reader has payload for any of the
+   * documents belonging to the {@link Term} given to the constructor.
+   */
+  public boolean setNextReader(AtomicReaderContext context) throws IOException {
     hasMore = false;
-    while (leaves.hasNext()) {
-      AtomicReaderContext ctx = leaves.next();
-      curDocBase = ctx.docBase;
-      Fields fields = ctx.reader().fields();
-      if (fields != null) {
-        Terms terms = fields.terms(term.field());
-        if (terms != null) {
-          reuseTE = terms.iterator(reuseTE);
-          if (reuseTE.seekExact(term.bytes(), true)) {
-            // this class is usually used to iterate on whatever a Query matched
-            // if it didn't match deleted documents, we won't receive them. if it
-            // did, we should iterate on them too, therefore we pass liveDocs=null
-            currentDPE = reuseTE.docsAndPositions(null, currentDPE, DocsAndPositionsEnum.FLAG_PAYLOADS);
-            if (currentDPE != null && (curDocID = currentDPE.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-              hasMore = true;
-              break;
-            }
+    Fields fields = context.reader().fields();
+    if (fields != null) {
+      Terms terms = fields.terms(term.field());
+      if (terms != null) {
+        reuseTE = terms.iterator(reuseTE);
+        if (reuseTE.seekExact(term.bytes(), true)) {
+          // this class is usually used to iterate on whatever a Query matched
+          // if it didn't match deleted documents, we won't receive them. if it
+          // did, we should iterate on them too, therefore we pass liveDocs=null
+          dpe = reuseTE.docsAndPositions(null, dpe, DocsAndPositionsEnum.FLAG_PAYLOADS);
+          if (dpe != null && (curDocID = dpe.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+            hasMore = true;
           }
         }
       }
     }
-  }
-  
-  /**
-   * Initialize the iterator. Should be done before the first call to
-   * {@link #getPayload(int)}. Returns {@code false} if no category list is
-   * found, or the category list has no documents.
-   */
-  public boolean init() throws IOException {
-    nextSegment();
     return hasMore;
   }
-
+  
   /**
    * Returns the {@link BytesRef payload} of the given document, or {@code null}
    * if the document does not exist, there are no more documents in the posting
-   * list, or the document exists but has not payload. You should call
-   * {@link #init()} before the first call to this method.
+   * list, or the document exists but has not payload. The given document IDs
+   * are treated as local to the reader given to
+   * {@link #setNextReader(AtomicReaderContext)}.
    */
   public BytesRef getPayload(int docID) throws IOException {
     if (!hasMore) {
       return null;
     }
 
-    // re-basing docId->localDocID is done fewer times than currentDoc->globalDoc
-    int localDocID = docID - curDocBase;
-    
-    if (curDocID > localDocID) {
+    if (curDocID > docID) {
       // document does not exist
       return null;
     }
     
-    if (curDocID < localDocID) {
-      // look for the document either in that segment, or others
-      while (hasMore && (curDocID = currentDPE.advance(localDocID)) == DocIdSetIterator.NO_MORE_DOCS) {
-        nextSegment(); // also updates curDocID
-        localDocID = docID - curDocBase;
-        // nextSegment advances to nextDoc, so check if we still need to advance
-        if (curDocID >= localDocID) {
-          break;
+    if (curDocID < docID) {
+      curDocID = dpe.advance(docID);
+      if (curDocID != docID) { // requested document does not have a payload
+        if (curDocID == DocIdSetIterator.NO_MORE_DOCS) { // no more docs in this reader
+          hasMore = false;
         }
-      }
-      
-      // we break from the above loop when:
-      // 1. we iterated over all segments (hasMore=false)
-      // 2. current segment advanced to a doc, either requested or higher
-      if (!hasMore || curDocID != localDocID) {
         return null;
       }
     }
 
     // we're on the document
-    assert currentDPE.freq() == 1 : "expecting freq=1 (got " + currentDPE.freq() + ") term=" + term + " doc=" + (curDocID + curDocBase);
-    int pos = currentDPE.nextPosition();
-    assert pos != -1 : "no positions for term=" + term + " doc=" + (curDocID + curDocBase);
-    return currentDPE.getPayload();
+    assert dpe.freq() == 1 : "expecting freq=1 (got " + dpe.freq() + ") term=" + term + " doc=" + curDocID;
+    int pos = dpe.nextPosition();
+    assert pos != -1 : "no positions for term=" + term + " doc=" + curDocID;
+    return dpe.getPayload();
   }
   
 }

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/ScoredDocIdCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/ScoredDocIdCollector.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/ScoredDocIdCollector.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/ScoredDocIdCollector.java Mon Jan 14 12:07:42 2013
@@ -62,7 +62,7 @@ public abstract class ScoredDocIdCollect
     }
 
     @Override
-    public ScoredDocIDsIterator scoredDocIdsIterator() {
+    protected ScoredDocIDsIterator scoredDocIdsIterator() {
       return new ScoredDocIDsIterator() {
 
         private DocIdSetIterator docIdsIter = docIds.iterator();
@@ -129,7 +129,7 @@ public abstract class ScoredDocIdCollect
     }
 
     @Override
-    public ScoredDocIDsIterator scoredDocIdsIterator() {
+    protected ScoredDocIDsIterator scoredDocIdsIterator() {
       return new ScoredDocIDsIterator() {
 
         private DocIdSetIterator docIdsIter = docIds.iterator();
@@ -189,8 +189,7 @@ public abstract class ScoredDocIdCollect
    *        do not require scoring, it is better to set it to <i>false</i>.
    */
   public static ScoredDocIdCollector create(int maxDoc, boolean enableScoring) {
-    return enableScoring   ? new ScoringDocIdCollector(maxDoc)
-                          : new NonScoringDocIdCollector(maxDoc);
+    return enableScoring ? new ScoringDocIdCollector(maxDoc) : new NonScoringDocIdCollector(maxDoc);
   }
 
   private ScoredDocIdCollector(int maxDoc) {
@@ -198,13 +197,14 @@ public abstract class ScoredDocIdCollect
     docIds = new FixedBitSet(maxDoc);
   }
 
+  protected abstract ScoredDocIDsIterator scoredDocIdsIterator() throws IOException;
+
   /** Returns the default score used when scoring is disabled. */
   public abstract float getDefaultScore();
 
   /** Set the default score. Only applicable if scoring is disabled. */
   public abstract void setDefaultScore(float defaultScore);
 
-  public abstract ScoredDocIDsIterator scoredDocIdsIterator() throws IOException;
 
   public ScoredDocIDs getScoredDocIDs() {
     return new ScoredDocIDs() {

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/StandardFacetsAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/StandardFacetsAccumulator.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/StandardFacetsAccumulator.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/StandardFacetsAccumulator.java Mon Jan 14 12:07:42 2013
@@ -4,22 +4,23 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map.Entry;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.util.IntsRef;
-
 import org.apache.lucene.facet.search.aggregator.Aggregator;
-import org.apache.lucene.facet.search.params.FacetSearchParams;
 import org.apache.lucene.facet.search.params.FacetRequest;
+import org.apache.lucene.facet.search.params.FacetSearchParams;
 import org.apache.lucene.facet.search.results.FacetResult;
 import org.apache.lucene.facet.search.results.IntermediateFacetResult;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 import org.apache.lucene.facet.util.PartitionsUtils;
 import org.apache.lucene.facet.util.ScoredDocIdsUtils;
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.util.IntsRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -179,11 +180,11 @@ public class StandardFacetsAccumulator e
       List<FacetResult> res = new ArrayList<FacetResult>();
       for (FacetRequest fr : searchParams.getFacetRequests()) {
         FacetResultsHandler frHndlr = fr.createFacetResultsHandler(taxonomyReader);
-        IntermediateFacetResult tmpResult = fr2tmpRes.get(fr); 
+        IntermediateFacetResult tmpResult = fr2tmpRes.get(fr);
         if (tmpResult == null) {
           continue; // do not add a null to the list.
         }
-        FacetResult facetRes = frHndlr.renderFacetResult(tmpResult); 
+        FacetResult facetRes = frHndlr.renderFacetResult(tmpResult);
         // final labeling if allowed (because labeling is a costly operation)
         if (isAllowLabeling()) {
           frHndlr.labelResult(facetRes);
@@ -213,18 +214,15 @@ public class StandardFacetsAccumulator e
 
   /** Check if it is worth to use complements */
   protected boolean shouldComplement(ScoredDocIDs docids) {
-    return 
-      mayComplement() && 
-      (docids.size() > indexReader.numDocs() * getComplementThreshold()) ;
+    return mayComplement() && (docids.size() > indexReader.numDocs() * getComplementThreshold()) ;
   }
 
   /**
    * Iterate over the documents for this partition and fill the facet arrays with the correct
    * count/complement count/value.
-   * @throws IOException If there is a low-level I/O error.
    */
-  private final void fillArraysForPartition(ScoredDocIDs docids,
-      FacetArrays facetArrays, int partition) throws IOException {
+  private final void fillArraysForPartition(ScoredDocIDs docids, FacetArrays facetArrays, int partition) 
+      throws IOException {
     
     if (isUsingComplements) {
       initArraysByTotalCounts(facetArrays, partition, docids.size());
@@ -236,27 +234,41 @@ public class StandardFacetsAccumulator e
 
     IntsRef ordinals = new IntsRef(32); // a reasonable start capacity for most common apps
     for (Entry<CategoryListIterator, Aggregator> entry : categoryLists.entrySet()) {
-      CategoryListIterator categoryList = entry.getKey();
-      if (!categoryList.init()) {
-        continue;
-      }
-
-      Aggregator categorator = entry.getValue();
-      ScoredDocIDsIterator iterator = docids.iterator();
+      final ScoredDocIDsIterator iterator = docids.iterator();
+      final CategoryListIterator categoryListIter = entry.getKey();
+      final Aggregator aggregator = entry.getValue();
+      Iterator<AtomicReaderContext> contexts = indexReader.leaves().iterator();
+      AtomicReaderContext current = null;
+      int maxDoc = -1;
       while (iterator.next()) {
         int docID = iterator.getDocID();
-        categoryList.getOrdinals(docID, ordinals);
+        while (docID >= maxDoc) { // find the segment which contains this document
+          if (!contexts.hasNext()) {
+            throw new RuntimeException("ScoredDocIDs contains documents outside this reader's segments !?");
+          }
+          current = contexts.next();
+          maxDoc = current.docBase + current.reader().maxDoc();
+          if (docID < maxDoc) { // segment has docs, check if it has categories
+            boolean validSegment = categoryListIter.setNextReader(current);
+            validSegment &= aggregator.setNextReader(current);
+            if (!validSegment) { // if categoryList or aggregtor say it's an invalid segment, skip all docs
+              while (docID < maxDoc && iterator.next()) {
+                docID = iterator.getDocID();
+              }
+            }
+          }
+        }
+        docID -= current.docBase;
+        categoryListIter.getOrdinals(docID, ordinals);
         if (ordinals.length == 0) {
-          continue;
+          continue; // document does not have category ordinals
         }
-        categorator.aggregate(docID, iterator.getScore(), ordinals);
+        aggregator.aggregate(docID, iterator.getScore(), ordinals);
       }
     }
   }
 
-  /**
-   * Init arrays for partition by total counts, optionally applying a factor
-   */
+  /** Init arrays for partition by total counts, optionally applying a factor */
   private final void initArraysByTotalCounts(FacetArrays facetArrays, int partition, int nAccumulatedDocs) {
     int[] intArray = facetArrays.getIntArray();
     totalFacetCounts.fillTotalCountsForPartition(intArray, partition);
@@ -302,10 +314,9 @@ public class StandardFacetsAccumulator e
 
     for (FacetRequest facetRequest : searchParams.getFacetRequests()) {
       Aggregator categoryAggregator = facetRequest.createAggregator(
-          isUsingComplements, facetArrays, indexReader,  taxonomyReader);
+          isUsingComplements, facetArrays, taxonomyReader);
 
-      CategoryListIterator cli = 
-        facetRequest.createCategoryListIterator(indexReader, taxonomyReader, searchParams, partition);
+      CategoryListIterator cli = facetRequest.createCategoryListIterator(taxonomyReader, searchParams, partition);
       
       // get the aggregator
       Aggregator old = categoryLists.put(cli, categoryAggregator);

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/TotalFacetCounts.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/TotalFacetCounts.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/TotalFacetCounts.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/TotalFacetCounts.java Mon Jan 14 12:07:42 2013
@@ -170,7 +170,7 @@ public class TotalFacetCounts {
         Aggregator aggregator = new CountingAggregator(counts[partition]);
         HashMap<CategoryListIterator, Aggregator> map = new HashMap<CategoryListIterator, Aggregator>();
         for (CategoryListParams clp: facetIndexingParams.getAllCategoryListParams()) {
-          final CategoryListIterator cli = clIteraor(clCache, clp, indexReader, partition);
+          final CategoryListIterator cli = clIteraor(clCache, clp, partition);
           map.put(cli, aggregator);
         }
         return map;
@@ -181,14 +181,14 @@ public class TotalFacetCounts {
     return new TotalFacetCounts(taxonomy, facetIndexingParams, counts, CreationType.Computed);
   }
   
-  static CategoryListIterator clIteraor(CategoryListCache clCache, CategoryListParams clp, 
-      IndexReader indexReader, int partition) throws IOException {
+  static CategoryListIterator clIteraor(CategoryListCache clCache, CategoryListParams clp, int partition) 
+      throws IOException {
     if (clCache != null) {
       CategoryListData cld = clCache.get(clp);
       if (cld != null) {
         return cld.iterator(partition);
       }
     }
-    return clp.createCategoryListIterator(indexReader, partition);
+    return clp.createCategoryListIterator(partition);
   }
 }
\ No newline at end of file

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/Aggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/Aggregator.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/Aggregator.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/Aggregator.java Mon Jan 14 12:07:42 2013
@@ -2,6 +2,7 @@ package org.apache.lucene.facet.search.a
 
 import java.io.IOException;
 
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.util.IntsRef;
 
 /*
@@ -22,22 +23,23 @@ import org.apache.lucene.util.IntsRef;
  */
 
 /**
- * An Aggregator is the analogue of Lucene's Collector (see
- * {@link org.apache.lucene.search.Collector}), for processing the categories
- * belonging to a certain document. The Aggregator is responsible for doing
- * whatever it wishes with the categories it is fed, e.g., counting the number
- * of times that each category appears, or performing some computation on their
- * association values.
- * <P>
- * Much of the function of an Aggregator implementation is not described by this
- * interface. This includes the constructor and getter methods to retrieve the
- * results of the aggregation.
+ * Aggregates the categories of documents given to
+ * {@link #aggregate(int, float, IntsRef)}. Note that the document IDs are local
+ * to the reader given to {@link #setNextReader(AtomicReaderContext)}.
  * 
  * @lucene.experimental
  */
 public interface Aggregator {
 
   /**
+   * Sets the {@link AtomicReaderContext} for which
+   * {@link #aggregate(int, float, IntsRef)} calls will be made. If this method
+   * returns false, {@link #aggregate(int, float, IntsRef)} should not be called
+   * for this reader.
+   */
+  public boolean setNextReader(AtomicReaderContext context) throws IOException;
+  
+  /**
    * Aggregate the ordinals of the given document ID (and its score). The given
    * ordinals offset is always zero.
    */

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/CountingAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/CountingAggregator.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/CountingAggregator.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/CountingAggregator.java Mon Jan 14 12:07:42 2013
@@ -2,6 +2,7 @@ package org.apache.lucene.facet.search.a
 
 import java.io.IOException;
 
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.util.IntsRef;
 
 /*
@@ -57,4 +58,9 @@ public class CountingAggregator implemen
     return counterArray == null ? 0 : counterArray.hashCode();
   }
   
+  @Override
+  public boolean setNextReader(AtomicReaderContext context) throws IOException {
+    return true;
+  }
+  
 }

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/ScoringAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/ScoringAggregator.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/ScoringAggregator.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/ScoringAggregator.java Mon Jan 14 12:07:42 2013
@@ -2,6 +2,7 @@ package org.apache.lucene.facet.search.a
 
 import java.io.IOException;
 
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.util.IntsRef;
 
 /*
@@ -58,4 +59,9 @@ public class ScoringAggregator implement
     return hashCode;
   }
 
+  @Override
+  public boolean setNextReader(AtomicReaderContext context) throws IOException {
+    return true;
+  }
+  
 }

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationFloatSumAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationFloatSumAggregator.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationFloatSumAggregator.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationFloatSumAggregator.java Mon Jan 14 12:07:42 2013
@@ -6,7 +6,7 @@ import org.apache.lucene.facet.associati
 import org.apache.lucene.facet.associations.FloatAssociationsPayloadIterator;
 import org.apache.lucene.facet.index.params.CategoryListParams;
 import org.apache.lucene.facet.search.aggregator.Aggregator;
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.collections.IntToFloatMap;
 
@@ -39,13 +39,13 @@ public class AssociationFloatSumAggregat
   protected final float[] sumArray;
   protected final FloatAssociationsPayloadIterator associations;
 
-  public AssociationFloatSumAggregator(IndexReader reader, float[] sumArray) throws IOException {
-    this(CategoryListParams.DEFAULT_TERM.field(), reader, sumArray);
+  public AssociationFloatSumAggregator(float[] sumArray) throws IOException {
+    this(CategoryListParams.DEFAULT_TERM.field(), sumArray);
   }
   
-  public AssociationFloatSumAggregator(String field, IndexReader reader, float[] sumArray) throws IOException {
+  public AssociationFloatSumAggregator(String field, float[] sumArray) throws IOException {
     this.field = field;
-    associations = new FloatAssociationsPayloadIterator(reader, field, new CategoryFloatAssociation());
+    associations = new FloatAssociationsPayloadIterator(field, new CategoryFloatAssociation());
     this.sumArray = sumArray;
   }
 
@@ -76,4 +76,9 @@ public class AssociationFloatSumAggregat
     return field.hashCode();
   }
 
+  @Override
+  public boolean setNextReader(AtomicReaderContext context) throws IOException {
+    return associations.setNextReader(context);
+  }
+  
 }

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationIntSumAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationIntSumAggregator.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationIntSumAggregator.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationIntSumAggregator.java Mon Jan 14 12:07:42 2013
@@ -6,7 +6,7 @@ import org.apache.lucene.facet.associati
 import org.apache.lucene.facet.associations.IntAssociationsPayloadIterator;
 import org.apache.lucene.facet.index.params.CategoryListParams;
 import org.apache.lucene.facet.search.aggregator.Aggregator;
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.collections.IntToIntMap;
 
@@ -39,13 +39,13 @@ public class AssociationIntSumAggregator
   protected final int[] sumArray;
   protected final IntAssociationsPayloadIterator associations;
 
-  public AssociationIntSumAggregator(IndexReader reader, int[] sumArray) throws IOException {
-    this(CategoryListParams.DEFAULT_TERM.field(), reader, sumArray);
+  public AssociationIntSumAggregator(int[] sumArray) throws IOException {
+    this(CategoryListParams.DEFAULT_TERM.field(), sumArray);
   }
   
-  public AssociationIntSumAggregator(String field, IndexReader reader, int[] sumArray) throws IOException {
+  public AssociationIntSumAggregator(String field, int[] sumArray) throws IOException {
     this.field = field;
-    associations = new IntAssociationsPayloadIterator(reader, field, new CategoryIntAssociation());
+    associations = new IntAssociationsPayloadIterator(field, new CategoryIntAssociation());
     this.sumArray = sumArray;
   }
 
@@ -76,4 +76,9 @@ public class AssociationIntSumAggregator
     return field.hashCode();
   }
 
+  @Override
+  public boolean setNextReader(AtomicReaderContext context) throws IOException {
+    return associations.setNextReader(context);
+  }
+
 }

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/cache/CategoryListData.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/cache/CategoryListData.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/cache/CategoryListData.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/cache/CategoryListData.java Mon Jan 14 12:07:42 2013
@@ -6,6 +6,7 @@ import org.apache.lucene.facet.index.par
 import org.apache.lucene.facet.index.params.FacetIndexingParams;
 import org.apache.lucene.facet.search.CategoryListIterator;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.util.IntsRef;
 
@@ -56,25 +57,30 @@ public class CategoryListData {
   }
   
   /** Compute category list data for caching for faster iteration. */
-  CategoryListData(IndexReader reader, TaxonomyReader taxo, 
-      FacetIndexingParams iparams, CategoryListParams clp) throws IOException {
+  CategoryListData(IndexReader reader, TaxonomyReader taxo, FacetIndexingParams iparams, CategoryListParams clp) 
+      throws IOException {
   
-    final int maxDoc = reader.maxDoc();
-    int[][][]dpf  = new int[maxDoc][][];
+    int[][][]dpf  = new int[reader.maxDoc()][][];
     int numPartitions = (int)Math.ceil(taxo.getSize()/(double)iparams.getPartitionSize());
     IntsRef ordinals = new IntsRef(32);
     for (int part = 0; part < numPartitions; part++) {
-      CategoryListIterator cli = clp.createCategoryListIterator(reader, part);
-      if (cli.init()) {
-        for (int doc = 0; doc < maxDoc; doc++) {
-          cli.getOrdinals(doc, ordinals);
-          if (ordinals.length > 0) {
-            if (dpf[doc] == null) {
-              dpf[doc] = new int[numPartitions][];
-            }
-            dpf[doc][part] = new int[ordinals.length];
-            for (int i = 0; i < ordinals.length; i++) {
-              dpf[doc][part][i] = ordinals.ints[i];
+      for (AtomicReaderContext context : reader.leaves()) {
+        CategoryListIterator cli = clp.createCategoryListIterator(part);
+        if (cli.setNextReader(context)) {
+          final int maxDoc = context.reader().maxDoc();
+          for (int i = 0; i < maxDoc; i++) {
+            cli.getOrdinals(i, ordinals);
+            if (ordinals.length > 0) {
+              int doc = i + context.docBase;
+              if (dpf[doc] == null) {
+                dpf[doc] = new int[numPartitions][];
+              }
+              if (dpf[doc][part] == null) {
+                dpf[doc][part] = new int[ordinals.length];
+              }
+              for (int j = 0; j < ordinals.length; j++) {
+                dpf[doc][part][j] = ordinals.ints[j];
+              }
             }
           }
         }
@@ -93,6 +99,7 @@ public class CategoryListData {
   /** Internal: category list iterator over uncompressed category info in RAM */
   private static class RAMCategoryListIterator implements CategoryListIterator {
     
+    private int docBase;
     private final int part;
     private final int[][][] dpc;
     
@@ -102,13 +109,15 @@ public class CategoryListData {
     }
 
     @Override
-    public boolean init() throws IOException {
+    public boolean setNextReader(AtomicReaderContext context) throws IOException {
+      docBase = context.docBase;
       return dpc != null && dpc.length > part;
     }
-
+    
     @Override
     public void getOrdinals(int docID, IntsRef ints) throws IOException {
       ints.length = 0;
+      docID += docBase;
       if (dpc.length > docID && dpc[docID] != null && dpc[docID][part] != null) {
         if (ints.ints.length < dpc[docID][part].length) {
           ints.grow(dpc[docID][part].length);

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/params/CountFacetRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/params/CountFacetRequest.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/params/CountFacetRequest.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/params/CountFacetRequest.java Mon Jan 14 12:07:42 2013
@@ -1,7 +1,5 @@
 package org.apache.lucene.facet.search.params;
 
-import org.apache.lucene.index.IndexReader;
-
 import org.apache.lucene.facet.search.FacetArrays;
 import org.apache.lucene.facet.search.aggregator.Aggregator;
 import org.apache.lucene.facet.search.aggregator.ComplementCountingAggregator;
@@ -47,8 +45,7 @@ public class CountFacetRequest extends F
   }
 
   @Override
-  public Aggregator createAggregator(boolean useComplements,
-      FacetArrays arrays, IndexReader reader, TaxonomyReader taxonomy) {
+  public Aggregator createAggregator(boolean useComplements, FacetArrays arrays, TaxonomyReader taxonomy) {
     // we rely on that, if needed, result is cleared by arrays!
     int[] a = arrays.getIntArray();
     if (useComplements) {

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/params/FacetRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/params/FacetRequest.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/params/FacetRequest.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/params/FacetRequest.java Mon Jan 14 12:07:42 2013
@@ -2,8 +2,6 @@ package org.apache.lucene.facet.search.p
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
-
 import org.apache.lucene.facet.index.params.CategoryListParams;
 import org.apache.lucene.facet.search.CategoryListIterator;
 import org.apache.lucene.facet.search.FacetArrays;
@@ -11,8 +9,8 @@ import org.apache.lucene.facet.search.Fa
 import org.apache.lucene.facet.search.TopKFacetResultsHandler;
 import org.apache.lucene.facet.search.TopKInEachNodeHandler;
 import org.apache.lucene.facet.search.aggregator.Aggregator;
-import org.apache.lucene.facet.search.cache.CategoryListData;
 import org.apache.lucene.facet.search.cache.CategoryListCache;
+import org.apache.lucene.facet.search.cache.CategoryListData;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 
@@ -314,33 +312,29 @@ public abstract class FacetRequest imple
    *          computation.
    * @param arrays
    *          provider for facet arrays in use for current computation.
-   * @param indexReader
-   *          index reader in effect.
    * @param taxonomy
    *          reader of taxonomy in effect.
    * @throws IOException If there is a low-level I/O error.
    */
-  public abstract Aggregator createAggregator(boolean useComplements,
-      FacetArrays arrays, IndexReader indexReader,
-      TaxonomyReader taxonomy) throws IOException;
+  public abstract Aggregator createAggregator(boolean useComplements, FacetArrays arrays, TaxonomyReader taxonomy) 
+      throws IOException;
 
   /**
-   * Create the category list iterator for the specified partition.
-   * If a non null cache is provided which contains the required data, 
-   * use it for the iteration.
+   * Create the category list iterator for the specified partition. If a non
+   * null cache is provided which contains the required data, use it for the
+   * iteration.
    */
-  public CategoryListIterator createCategoryListIterator(IndexReader reader,
-      TaxonomyReader taxo, FacetSearchParams sParams, int partition)
+  public CategoryListIterator createCategoryListIterator(TaxonomyReader taxo, FacetSearchParams sParams, int partition)
       throws IOException {
     CategoryListCache clCache = sParams.getCategoryListCache();
     CategoryListParams clParams = sParams.getFacetIndexingParams().getCategoryListParams(categoryPath);
-    if (clCache!=null) {
+    if (clCache != null) {
       CategoryListData clData = clCache.get(clParams);
-      if (clData!=null) {
+      if (clData != null) {
         return clData.iterator(partition);
       }
     }
-    return clParams.createCategoryListIterator(reader, partition);
+    return clParams.createCategoryListIterator(partition);
   }
 
   /**

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/params/ScoreFacetRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/params/ScoreFacetRequest.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/params/ScoreFacetRequest.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/params/ScoreFacetRequest.java Mon Jan 14 12:07:42 2013
@@ -1,7 +1,5 @@
 package org.apache.lucene.facet.search.params;
 
-import org.apache.lucene.index.IndexReader;
-
 import org.apache.lucene.facet.search.FacetArrays;
 import org.apache.lucene.facet.search.aggregator.Aggregator;
 import org.apache.lucene.facet.search.aggregator.ScoringAggregator;
@@ -38,9 +36,7 @@ public class ScoreFacetRequest extends F
   }
 
   @Override
-  public Aggregator createAggregator(boolean useComplements,
-                                      FacetArrays arrays, IndexReader reader,
-                                      TaxonomyReader taxonomy) {
+  public Aggregator createAggregator(boolean useComplements, FacetArrays arrays, TaxonomyReader taxonomy) {
     assert !useComplements : "complements are not supported by this FacetRequest";
     return new ScoringAggregator(arrays.getFloatArray());
   }

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationFloatSumFacetRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationFloatSumFacetRequest.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationFloatSumFacetRequest.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationFloatSumFacetRequest.java Mon Jan 14 12:07:42 2013
@@ -2,8 +2,6 @@ package org.apache.lucene.facet.search.p
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
-
 import org.apache.lucene.facet.search.FacetArrays;
 import org.apache.lucene.facet.search.aggregator.Aggregator;
 import org.apache.lucene.facet.search.aggregator.associations.AssociationFloatSumAggregator;
@@ -45,10 +43,10 @@ public class AssociationFloatSumFacetReq
   }
 
   @Override
-  public Aggregator createAggregator(boolean useComplements, FacetArrays arrays, IndexReader reader, 
-      TaxonomyReader taxonomy) throws IOException {
+  public Aggregator createAggregator(boolean useComplements, FacetArrays arrays, TaxonomyReader taxonomy) 
+      throws IOException {
     assert !useComplements : "complements are not supported by this FacetRequest";
-    return new AssociationFloatSumAggregator(reader, arrays.getFloatArray());
+    return new AssociationFloatSumAggregator(arrays.getFloatArray());
   }
 
   @Override

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationIntSumFacetRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationIntSumFacetRequest.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationIntSumFacetRequest.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationIntSumFacetRequest.java Mon Jan 14 12:07:42 2013
@@ -2,8 +2,6 @@ package org.apache.lucene.facet.search.p
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
-
 import org.apache.lucene.facet.search.FacetArrays;
 import org.apache.lucene.facet.search.aggregator.Aggregator;
 import org.apache.lucene.facet.search.aggregator.associations.AssociationIntSumAggregator;
@@ -45,10 +43,10 @@ public class AssociationIntSumFacetReque
   }
 
   @Override
-  public Aggregator createAggregator(boolean useComplements, FacetArrays arrays, IndexReader reader, 
-      TaxonomyReader taxonomy) throws IOException {
+  public Aggregator createAggregator(boolean useComplements, FacetArrays arrays, TaxonomyReader taxonomy) 
+      throws IOException {
     assert !useComplements : "complements are not supported by this FacetRequest";
-    return new AssociationIntSumAggregator(reader, arrays.getIntArray());
+    return new AssociationIntSumAggregator(arrays.getIntArray());
   }
 
   @Override

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/sampling/Sampler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/sampling/Sampler.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/sampling/Sampler.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/sampling/Sampler.java Mon Jan 14 12:07:42 2013
@@ -60,6 +60,7 @@ public abstract class Sampler {
   
   /**
    * Construct with certain {@link SamplingParams}
+   * 
    * @param params sampling params in effect
    * @throws IllegalArgumentException if the provided SamplingParams are not valid 
    */
@@ -110,16 +111,15 @@ public abstract class Sampler {
    * @param sampleSetSize required size of sample set
    * @return sample of the input set in the required size
    */
-  protected abstract SampleResult createSample(ScoredDocIDs docids, int actualSize,
-      int sampleSetSize) throws IOException;
+  protected abstract SampleResult createSample(ScoredDocIDs docids, int actualSize, int sampleSetSize) 
+      throws IOException;
 
   /**
    * Get a fixer of sample facet accumulation results. Default implementation
    * returns a <code>TakmiSampleFixer</code> which is adequate only for
    * counting. For any other accumulator, provide a different fixer.
    */
-  public SampleFixer getSampleFixer(
-      IndexReader indexReader, TaxonomyReader taxonomyReader,
+  public SampleFixer getSampleFixer(IndexReader indexReader, TaxonomyReader taxonomyReader,
       FacetSearchParams searchParams) {
     return new TakmiSampleFixer(indexReader, taxonomyReader, searchParams);
   }
@@ -161,10 +161,10 @@ public abstract class Sampler {
     OverSampledFacetRequest sampledFreq = null;
     
     try {
-      sampledFreq = (OverSampledFacetRequest)facetResult.getFacetRequest();
+      sampledFreq = (OverSampledFacetRequest) facetResult.getFacetRequest();
     } catch (ClassCastException e) {
       throw new IllegalArgumentException(
-          "It is only valid to call this method with result obtained for a" +
+          "It is only valid to call this method with result obtained for a " +
           "facet request created through sampler.overSamlpingSearchParams()",
           e);
     }
@@ -215,19 +215,15 @@ public abstract class Sampler {
     }
     
     @Override
-    public CategoryListIterator createCategoryListIterator(IndexReader reader,
-        TaxonomyReader taxo, FacetSearchParams sParams, int partition)
-        throws IOException {
-      return orig.createCategoryListIterator(reader, taxo, sParams, partition);
+    public CategoryListIterator createCategoryListIterator(TaxonomyReader taxo, FacetSearchParams sParams, 
+        int partition) throws IOException {
+      return orig.createCategoryListIterator(taxo, sParams, partition);
     }
-
     
     @Override
-    public Aggregator createAggregator(boolean useComplements,
-        FacetArrays arrays, IndexReader indexReader,
-        TaxonomyReader taxonomy) throws IOException {
-      return orig.createAggregator(useComplements, arrays, indexReader,
-          taxonomy);
+    public Aggregator createAggregator(boolean useComplements, FacetArrays arrays, TaxonomyReader taxonomy) 
+        throws IOException {
+      return orig.createAggregator(useComplements, arrays, taxonomy);
     }
 
     @Override
@@ -245,4 +241,5 @@ public abstract class Sampler {
       return orig.supportsComplements();
     }
   }
+
 }

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/sampling/TakmiSampleFixer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/sampling/TakmiSampleFixer.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/sampling/TakmiSampleFixer.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/sampling/TakmiSampleFixer.java Mon Jan 14 12:07:42 2013
@@ -91,8 +91,7 @@ class TakmiSampleFixer implements Sample
    *          full set of matching documents.
    * @throws IOException If there is a low-level I/O error.
    */
-  private void recount(FacetResultNode fresNode, ScoredDocIDs docIds)
-      throws IOException {
+  private void recount(FacetResultNode fresNode, ScoredDocIDs docIds) throws IOException {
     // TODO (Facet): change from void to return the new, smaller docSet, and use
     // that for the children, as this will make their intersection ops faster.
     // can do this only when the new set is "sufficiently" smaller.
@@ -109,8 +108,7 @@ class TakmiSampleFixer implements Sample
     Bits liveDocs = MultiFields.getLiveDocs(indexReader);
     int updatedCount = countIntersection(MultiFields.getTermDocsEnum(indexReader, liveDocs,
                                                                      drillDownTerm.field(), drillDownTerm.bytes(),
-                                                                     0),
-                                         docIds.iterator());
+                                                                     0), docIds.iterator());
 
     fresNode.setValue(updatedCount);
   }

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/util/MultiCategoryListIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/util/MultiCategoryListIterator.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/util/MultiCategoryListIterator.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/util/MultiCategoryListIterator.java Mon Jan 14 12:07:42 2013
@@ -5,6 +5,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.lucene.facet.search.CategoryListIterator;
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.util.IntsRef;
 
 /*
@@ -42,9 +43,10 @@ public class MultiCategoryListIterator i
   }
 
   @Override
-  public boolean init() throws IOException {
+  public boolean setNextReader(AtomicReaderContext context) throws IOException {
+    validIterators.clear();
     for (CategoryListIterator cli : iterators) {
-      if (cli.init()) {
+      if (cli.setNextReader(context)) {
         validIterators.add(cli);
       }
     }

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/util/ScoredDocIdsUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/util/ScoredDocIdsUtils.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/util/ScoredDocIdsUtils.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/util/ScoredDocIdsUtils.java Mon Jan 14 12:07:42 2013
@@ -3,17 +3,18 @@ package org.apache.lucene.facet.util;
 import java.io.IOException;
 import java.util.Arrays;
 
+import org.apache.lucene.facet.search.ScoredDocIDs;
+import org.apache.lucene.facet.search.ScoredDocIDsIterator;
+import org.apache.lucene.index.AtomicReader;
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.OpenBitSet;
+import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.OpenBitSetDISI;
 
-import org.apache.lucene.facet.search.ScoredDocIDs;
-import org.apache.lucene.facet.search.ScoredDocIDsIterator;
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -49,48 +50,57 @@ public class ScoredDocIdsUtils {
    * @param reader holding the number of documents & information about deletions.
    */
   public final static ScoredDocIDs getComplementSet(final ScoredDocIDs docids, final IndexReader reader)
-  throws IOException {
+      throws IOException {
     final int maxDoc = reader.maxDoc();
 
     DocIdSet docIdSet = docids.getDocIDs();
-    final OpenBitSet complement;
-    if (docIdSet instanceof OpenBitSet) {
+    final FixedBitSet complement;
+    if (docIdSet instanceof FixedBitSet) {
       // That is the most common case, if ScoredDocIdsCollector was used.
-      complement = ((OpenBitSet) docIdSet).clone();
+      complement = ((FixedBitSet) docIdSet).clone();
     } else {
-      complement = new OpenBitSetDISI(docIdSet.iterator(), maxDoc);
+      complement = new FixedBitSet(maxDoc);
+      DocIdSetIterator iter = docIdSet.iterator();
+      int doc;
+      while ((doc = iter.nextDoc()) < maxDoc) {
+        complement.set(doc);
+      }
     }
-
     complement.flip(0, maxDoc);
-
-    // Remove all Deletions from the complement set
     clearDeleted(reader, complement);
 
     return createScoredDocIds(complement, maxDoc);
   }
-
-  /**
-   * Clear all deleted documents from a given open-bit-set according to a given reader 
-   */
-  private static void clearDeleted(final IndexReader reader, 
-      final OpenBitSet set) throws IOException {
-
+  
+  /** Clear all deleted documents from a given open-bit-set according to a given reader */
+  private static void clearDeleted(final IndexReader reader, final FixedBitSet set) throws IOException {
+    
     // If there are no deleted docs
     if (!reader.hasDeletions()) {
       return; // return immediately
     }
     
-    Bits bits = MultiFields.getLiveDocs(reader);
-
     DocIdSetIterator it = set.iterator();
-    int doc = DocIdSetIterator.NO_MORE_DOCS;
-    while ((doc = it.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-      if (!bits.get(doc)) {
-        set.fastClear(doc);
+    int doc = it.nextDoc(); 
+    for (AtomicReaderContext context : reader.leaves()) {
+      AtomicReader r = context.reader();
+      final int maxDoc = r.maxDoc() + context.docBase;
+      if (doc >= maxDoc) { // skip this segment
+        continue;
+      }
+      if (!r.hasDeletions()) { // skip all docs that belong to this reader as it has no deletions
+        while ((doc = it.nextDoc()) < maxDoc) {}
+        continue;
       }
+      Bits liveDocs = r.getLiveDocs();
+      do {
+        if (!liveDocs.get(doc - context.docBase)) {
+          set.clear(doc);
+        }
+      } while ((doc = it.nextDoc()) < maxDoc);
     }
   }
-
+  
   /**
    * Create a subset of an existing ScoredDocIDs object.
    * 
@@ -274,8 +284,7 @@ public class ScoredDocIdsUtils {
               if (target <= next) {
                 target = next + 1;
               }
-              return next = target >= maxDoc ? NO_MORE_DOCS
-                  : target;
+              return next = target >= maxDoc ? NO_MORE_DOCS : target;
             }
 
             @Override
@@ -420,4 +429,5 @@ public class ScoredDocIdsUtils {
       }
     }
   }
+  
 }
\ No newline at end of file

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/FacetTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/FacetTestBase.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/FacetTestBase.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/FacetTestBase.java Mon Jan 14 12:07:42 2013
@@ -317,8 +317,7 @@ public abstract class FacetTestBase exte
   }
   
   /** Validate results equality */
-  protected static void assertSameResults(List<FacetResult> expected,
-                                          List<FacetResult> actual) {
+  protected static void assertSameResults(List<FacetResult> expected, List<FacetResult> actual) {
     String expectedResults = resStringValueOnly(expected);
     String actualResults = resStringValueOnly(actual);
     if (!expectedResults.equals(actualResults)) {

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/AdaptiveAccumulatorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/AdaptiveAccumulatorTest.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/AdaptiveAccumulatorTest.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/AdaptiveAccumulatorTest.java Mon Jan 14 12:07:42 2013
@@ -29,12 +29,11 @@ import org.apache.lucene.facet.taxonomy.
 public class AdaptiveAccumulatorTest extends BaseSampleTestTopK {
 
   @Override
-  protected FacetsAccumulator getSamplingAccumulator(Sampler sampler,
-      TaxonomyReader taxoReader, IndexReader indexReader,
-      FacetSearchParams searchParams) {
-    AdaptiveFacetsAccumulator res = new AdaptiveFacetsAccumulator(searchParams,
-        indexReader, taxoReader);
+  protected FacetsAccumulator getSamplingAccumulator(Sampler sampler, TaxonomyReader taxoReader, 
+      IndexReader indexReader, FacetSearchParams searchParams) {
+    AdaptiveFacetsAccumulator res = new AdaptiveFacetsAccumulator(searchParams, indexReader, taxoReader);
     res.setSampler(sampler);
     return res;
   }
+  
 }

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/CategoryListIteratorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/CategoryListIteratorTest.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/CategoryListIteratorTest.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/CategoryListIteratorTest.java Mon Jan 14 12:07:42 2013
@@ -14,6 +14,7 @@ import org.apache.lucene.analysis.tokena
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
@@ -106,30 +107,31 @@ public class CategoryListIteratorTest ex
     IndexReader reader = writer.getReader();
     writer.close();
 
-    IntsRef ordinals = new IntsRef();
-    CategoryListIterator cli = new PayloadCategoryListIteraor(reader, new Term("f","1"), encoder.createMatchingDecoder());
-    cli.init();
     int totalCategories = 0;
-    for (int i = 0; i < data.length; i++) {
-      Set<Integer> values = new HashSet<Integer>();
-      for (int j = 0; j < data[i].length; j++) {
-        values.add(data[i].ints[j]);
-      }
-      cli.getOrdinals(i, ordinals);
-      assertTrue("no ordinals for document " + i, ordinals.length > 0);
-      for (int j = 0; j < ordinals.length; j++) {
-        assertTrue("expected category not found: " + ordinals.ints[j], values.contains(ordinals.ints[j]));
+    IntsRef ordinals = new IntsRef();
+    CategoryListIterator cli = new PayloadCategoryListIteraor(new Term("f","1"), encoder.createMatchingDecoder());
+    for (AtomicReaderContext context : reader.leaves()) {
+      cli.setNextReader(context);
+      int maxDoc = context.reader().maxDoc();
+      int dataIdx = context.docBase;
+      for (int doc = 0; doc < maxDoc; doc++, dataIdx++) {
+        Set<Integer> values = new HashSet<Integer>();
+        for (int j = 0; j < data[dataIdx].length; j++) {
+          values.add(data[dataIdx].ints[j]);
+        }
+        cli.getOrdinals(doc, ordinals);
+        assertTrue("no ordinals for document " + doc, ordinals.length > 0);
+        for (int j = 0; j < ordinals.length; j++) {
+          assertTrue("expected category not found: " + ordinals.ints[j], values.contains(ordinals.ints[j]));
+        }
+        totalCategories += ordinals.length;
       }
-      totalCategories += ordinals.length;
     }
-    assertEquals("Missing categories!",10,totalCategories);
+    assertEquals("Missing categories!", 10, totalCategories);
     reader.close();
     dir.close();
   }
 
-  /**
-   * Test that a document with no payloads does not confuse the payload decoder.
-   */
   @Test
   public void testPayloadIteratorWithInvalidDoc() throws Exception {
     Directory dir = newDirectory();
@@ -160,24 +162,28 @@ public class CategoryListIteratorTest ex
     IndexReader reader = writer.getReader();
     writer.close();
 
-    IntsRef ordinals = new IntsRef();
-    CategoryListIterator cli = new PayloadCategoryListIteraor(reader, new Term("f","1"), encoder.createMatchingDecoder());
-    assertTrue("Failed to initialize payload iterator", cli.init());
     int totalCategories = 0;
-    for (int i = 0; i < data.length; i++) {
-      Set<Integer> values = new HashSet<Integer>();
-      for (int j = 0; j < data[i].length; j++) {
-        values.add(data[i].ints[j]);
-      }
-      cli.getOrdinals(i, ordinals);
-      if (i == 0) {
-        assertTrue("document 0 must have a payload", ordinals.length > 0);
-        for (int j = 0; j < ordinals.length; j++) {
-          assertTrue("expected category not found: " + ordinals.ints[j], values.contains(ordinals.ints[j]));
+    IntsRef ordinals = new IntsRef();
+    CategoryListIterator cli = new PayloadCategoryListIteraor(new Term("f","1"), encoder.createMatchingDecoder());
+    for (AtomicReaderContext context : reader.leaves()) {
+      cli.setNextReader(context);
+      int maxDoc = context.reader().maxDoc();
+      int dataIdx = context.docBase;
+      for (int doc = 0; doc < maxDoc; doc++, dataIdx++) {
+        Set<Integer> values = new HashSet<Integer>();
+        for (int j = 0; j < data[dataIdx].length; j++) {
+          values.add(data[dataIdx].ints[j]);
+        }
+        cli.getOrdinals(doc, ordinals);
+        if (dataIdx == 0) {
+          assertTrue("document 0 must have a payload", ordinals.length > 0);
+          for (int j = 0; j < ordinals.length; j++) {
+            assertTrue("expected category not found: " + ordinals.ints[j], values.contains(ordinals.ints[j]));
+          }
+          totalCategories += ordinals.length;
+        } else {
+          assertTrue("only document 0 should have a payload", ordinals.length == 0);
         }
-        totalCategories += ordinals.length;
-      } else {
-        assertTrue("only document 0 should have a payload", ordinals.length == 0);
       }
     }
     assertEquals("Wrong number of total categories!", 2, totalCategories);

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/TestCategoryListCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/TestCategoryListCache.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/TestCategoryListCache.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/TestCategoryListCache.java Mon Jan 14 12:07:42 2013
@@ -22,6 +22,7 @@ import org.apache.lucene.facet.search.pa
 import org.apache.lucene.facet.search.params.FacetSearchParams;
 import org.apache.lucene.facet.search.results.FacetResult;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
+import org.apache.lucene.index.AtomicReaderContext;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -132,8 +133,8 @@ public class TestCategoryListCache exten
             }
           }
           @Override
-          public boolean init() throws IOException {
-            return it.init();
+          public boolean setNextReader(AtomicReaderContext context) throws IOException {
+            return it.setNextReader(context);
           }
         };
       }

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/params/MultiCategoryListIteratorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/params/MultiCategoryListIteratorTest.java?rev=1432894&r1=1432893&r2=1432894&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/params/MultiCategoryListIteratorTest.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/params/MultiCategoryListIteratorTest.java Mon Jan 14 12:07:42 2013
@@ -17,6 +17,7 @@ import org.apache.lucene.facet.taxonomy.
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
 import org.apache.lucene.facet.util.MultiCategoryListIterator;
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.Term;
@@ -100,21 +101,24 @@ public class MultiCategoryListIteratorTe
         clCache.loadAndRegister(clp, indexReader, taxoReader, indexingParams);
         iterators[i] = clCache.get(clp).iterator(0); // no partitions
       } else {
-        iterators[i] = new PayloadCategoryListIteraor(indexReader, clp.getTerm(), decoder);
+        iterators[i] = new PayloadCategoryListIteraor(clp.getTerm(), decoder);
       }
     }
     MultiCategoryListIterator cli = new MultiCategoryListIterator(iterators);
-    assertTrue("failed to init multi-iterator", cli.init());
-    IntsRef ordinals = new IntsRef();
-    int maxDoc = indexReader.maxDoc();
-    for (int i = 0; i < maxDoc; i++) {
-      cli.getOrdinals(i, ordinals);
-      assertTrue("document " + i + " does not have categories", ordinals.length > 0);
-      for (int j = 0; j < ordinals.length; j++) {
-        CategoryPath cp = taxoReader.getPath(ordinals.ints[j]);
-        assertNotNull("ordinal " + ordinals.ints[j] + " not found in taxonomy", cp);
-        if (cp.length == 2) {
-          assertEquals("invalid category for document " + i, i, Integer.parseInt(cp.components[1]));
+    for (AtomicReaderContext context : indexReader.leaves()) {
+      assertTrue("failed to init multi-iterator", cli.setNextReader(context));
+      IntsRef ordinals = new IntsRef();
+      final int maxDoc = context.reader().maxDoc();
+      for (int i = 0; i < maxDoc; i++) {
+        cli.getOrdinals(i, ordinals);
+        assertTrue("document " + i + " does not have categories", ordinals.length > 0);
+        for (int j = 0; j < ordinals.length; j++) {
+          CategoryPath cp = taxoReader.getPath(ordinals.ints[j]);
+          assertNotNull("ordinal " + ordinals.ints[j] + " not found in taxonomy", cp);
+          if (cp.length == 2) {
+            int globalDoc = i + context.docBase;
+            assertEquals("invalid category for document " + globalDoc, globalDoc, Integer.parseInt(cp.components[1]));
+          }
         }
       }
     }