You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2015/07/06 19:16:37 UTC

svn commit: r1689462 - in /lucene/dev/branches/branch_5x: ./ lucene/ lucene/highlighter/ lucene/highlighter/src/test/org/apache/lucene/search/highlight/ lucene/join/ lucene/join/src/java/org/apache/lucene/search/join/ lucene/join/src/test/org/apache/lu...

Author: jpountz
Date: Mon Jul  6 17:16:36 2015
New Revision: 1689462

URL: http://svn.apache.org/r1689462
Log:
LUCENE-6649: Remove dependency of lucene/join on Filter.

Added:
    lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/BitSetProducer.java
      - copied unchanged from r1689432, lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/BitSetProducer.java
    lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/QueryBitSetProducer.java
      - copied, changed from r1689432, lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/QueryBitSetProducer.java
Modified:
    lucene/dev/branches/branch_5x/   (props changed)
    lucene/dev/branches/branch_5x/lucene/   (props changed)
    lucene/dev/branches/branch_5x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_5x/lucene/highlighter/   (props changed)
    lucene/dev/branches/branch_5x/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
    lucene/dev/branches/branch_5x/lucene/join/   (props changed)
    lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/BitDocIdSetCachingWrapperFilter.java
    lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/BitDocIdSetFilter.java
    lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java
    lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java
    lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinSortField.java
    lucene/dev/branches/branch_5x/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java
    lucene/dev/branches/branch_5x/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinSorting.java
    lucene/dev/branches/branch_5x/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinValidation.java
    lucene/dev/branches/branch_5x/solr/   (props changed)
    lucene/dev/branches/branch_5x/solr/contrib/   (props changed)
    lucene/dev/branches/branch_5x/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestHierarchicalDocBuilder.java
    lucene/dev/branches/branch_5x/solr/core/   (props changed)
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/response/transform/ChildDocTransformerFactory.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/join/BlockJoinChildQParser.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/join/BlockJoinParentQParser.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/update/AddBlockUpdateTest.java

Modified: lucene/dev/branches/branch_5x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/CHANGES.txt?rev=1689462&r1=1689461&r2=1689462&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/lucene/CHANGES.txt Mon Jul  6 17:16:36 2015
@@ -149,6 +149,10 @@ API Changes
 * LUCENE-6646: Make EarlyTerminatingCollector take a Sort object directly
   instead of a SortingMergePolicy. (Christine Poerschke via Adrien Grand)
 
+* LUCENE-6649: BitDocIdSetFilter and BitDocIdSetCachingWrapperFilter are now
+  deprecated in favour of BitSetProducer and QueryBitSetProducer, which do not
+  extend oal.search.Filter. (Adrien Grand)
+
 Bug fixes
 
 * LUCENE-6500: ParallelCompositeReader did not always call

Modified: lucene/dev/branches/branch_5x/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java?rev=1689462&r1=1689461&r2=1689462&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java (original)
+++ lucene/dev/branches/branch_5x/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java Mon Jul  6 17:16:36 2015
@@ -76,8 +76,8 @@ import org.apache.lucene.search.TermRang
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.search.WildcardQuery;
 import org.apache.lucene.search.highlight.SynonymTokenizer.TestHighlightRunner;
-import org.apache.lucene.search.join.BitDocIdSetCachingWrapperFilter;
-import org.apache.lucene.search.join.BitDocIdSetFilter;
+import org.apache.lucene.search.join.QueryBitSetProducer;
+import org.apache.lucene.search.join.BitSetProducer;
 import org.apache.lucene.search.join.ScoreMode;
 import org.apache.lucene.search.join.ToChildBlockJoinQuery;
 import org.apache.lucene.search.join.ToParentBlockJoinQuery;
@@ -601,7 +601,7 @@ public class HighlighterTest extends Bas
   }
   
   public void testToParentBlockJoinQuery() throws Exception {
-    BitDocIdSetFilter parentFilter = new BitDocIdSetCachingWrapperFilter(
+    BitSetProducer parentFilter = new QueryBitSetProducer(
         new QueryWrapperFilter(
           new TermQuery(new Term(FIELD_NAME, "parent"))));
     
@@ -627,7 +627,7 @@ public class HighlighterTest extends Bas
   }
   
   public void testToChildBlockJoinQuery() throws Exception {
-    BitDocIdSetFilter parentFilter = new BitDocIdSetCachingWrapperFilter(
+    BitSetProducer parentFilter = new QueryBitSetProducer(
         new QueryWrapperFilter(
           new TermQuery(new Term(FIELD_NAME, "parent"))));
     

Modified: lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/BitDocIdSetCachingWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/BitDocIdSetCachingWrapperFilter.java?rev=1689462&r1=1689461&r2=1689462&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/BitDocIdSetCachingWrapperFilter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/BitDocIdSetCachingWrapperFilter.java Mon Jul  6 17:16:36 2015
@@ -31,6 +31,7 @@ import org.apache.lucene.util.BitDocIdSe
 
 /**
  * {@link Filter} wrapper that implements {@link BitDocIdSetFilter}.
+ * @deprecated Use {@link QueryBitSetProducer} instead
  */
 public class BitDocIdSetCachingWrapperFilter extends BitDocIdSetFilter {
   private final Filter filter;

Modified: lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/BitDocIdSetFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/BitDocIdSetFilter.java?rev=1689462&r1=1689461&r2=1689462&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/BitDocIdSetFilter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/BitDocIdSetFilter.java Mon Jul  6 17:16:36 2015
@@ -18,18 +18,21 @@ package org.apache.lucene.search.join;
  */
 
 import java.io.IOException;
+import java.util.Objects;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.BitsFilteredDocIdSet;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.Filter;
 import org.apache.lucene.util.BitDocIdSet;
+import org.apache.lucene.util.BitSet;
 import org.apache.lucene.util.Bits;
 
 /**
  * A {@link Filter} that produces {@link BitDocIdSet}s.
+ * @deprecated Use {@link BitSetProducer} instead
  */
-public abstract class BitDocIdSetFilter extends Filter {
+public abstract class BitDocIdSetFilter extends Filter implements BitSetProducer {
 
   /** Sole constructor, typically called from sub-classes. */
   protected BitDocIdSetFilter() {}
@@ -45,4 +48,15 @@ public abstract class BitDocIdSetFilter
     return BitsFilteredDocIdSet.wrap(getDocIdSet(context), acceptDocs);
   }
 
+  @Override
+  public final BitSet getBitSet(LeafReaderContext context) throws IOException {
+    final BitDocIdSet set = getDocIdSet(context);
+    if (set == null) {
+      return null;
+    } else {
+      final BitSet bits = set.bits();
+      return Objects.requireNonNull(bits);
+    }
+  }
+
 }

Copied: lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/QueryBitSetProducer.java (from r1689432, lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/QueryBitSetProducer.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/QueryBitSetProducer.java?p2=lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/QueryBitSetProducer.java&p1=lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/QueryBitSetProducer.java&r1=1689432&r2=1689462&rev=1689462&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/QueryBitSetProducer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/QueryBitSetProducer.java Mon Jul  6 17:16:36 2015
@@ -40,7 +40,7 @@ import org.apache.lucene.util.BitSet;
  */
 public class QueryBitSetProducer implements BitSetProducer {
   private final Query query;
-  private final Map<Object,DocIdSet> cache = Collections.synchronizedMap(new WeakHashMap<>());
+  private final Map<Object,DocIdSet> cache = Collections.synchronizedMap(new WeakHashMap<Object,DocIdSet>());
 
   /** Wraps another query's result and caches it into bitsets.
    * @param query Query to cache results of

Modified: lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java?rev=1689462&r1=1689461&r2=1689462&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java Mon Jul  6 17:16:36 2015
@@ -31,7 +31,6 @@ import org.apache.lucene.search.IndexSea
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
-import org.apache.lucene.util.BitDocIdSet;
 import org.apache.lucene.util.BitSet;
 
 /**
@@ -51,7 +50,7 @@ public class ToChildBlockJoinQuery exten
   static final String INVALID_QUERY_MESSAGE = "Parent query yields document which is not matched by parents filter, docID=";
   static final String ILLEGAL_ADVANCE_ON_PARENT = "Expect to be advanced on child docs only. got docID=";
 
-  private final BitDocIdSetFilter parentsFilter;
+  private final BitSetProducer parentsFilter;
   private final Query parentQuery;
 
   // If we are rewritten, this is the original parentQuery we
@@ -67,14 +66,14 @@ public class ToChildBlockJoinQuery exten
    * @param parentQuery Query that matches parent documents
    * @param parentsFilter Filter identifying the parent documents.
    */
-  public ToChildBlockJoinQuery(Query parentQuery, BitDocIdSetFilter parentsFilter) {
+  public ToChildBlockJoinQuery(Query parentQuery, BitSetProducer parentsFilter) {
     super();
     this.origParentQuery = parentQuery;
     this.parentQuery = parentQuery;
     this.parentsFilter = parentsFilter;
   }
 
-  private ToChildBlockJoinQuery(Query origParentQuery, Query parentQuery, BitDocIdSetFilter parentsFilter) {
+  private ToChildBlockJoinQuery(Query origParentQuery, Query parentQuery, BitSetProducer parentsFilter) {
     super();
     this.origParentQuery = origParentQuery;
     this.parentQuery = parentQuery;
@@ -94,10 +93,10 @@ public class ToChildBlockJoinQuery exten
   private static class ToChildBlockJoinWeight extends Weight {
     private final Query joinQuery;
     private final Weight parentWeight;
-    private final BitDocIdSetFilter parentsFilter;
+    private final BitSetProducer parentsFilter;
     private final boolean doScores;
 
-    public ToChildBlockJoinWeight(Query joinQuery, Weight parentWeight, BitDocIdSetFilter parentsFilter, boolean doScores) {
+    public ToChildBlockJoinWeight(Query joinQuery, Weight parentWeight, BitSetProducer parentsFilter, boolean doScores) {
       super(joinQuery);
       this.joinQuery = joinQuery;
       this.parentWeight = parentWeight;
@@ -132,13 +131,13 @@ public class ToChildBlockJoinQuery exten
 
       // NOTE: this doesn't take acceptDocs into account, the responsibility
       // to not match deleted docs is on the scorer
-      final BitDocIdSet parents = parentsFilter.getDocIdSet(readerContext);
+      final BitSet parents = parentsFilter.getBitSet(readerContext);
       if (parents == null) {
         // No parents
         return null;
       }
 
-      return new ToChildBlockJoinScorer(this, parentScorer, parents.bits(), doScores);
+      return new ToChildBlockJoinScorer(this, parentScorer, parents, doScores);
     }
 
     @Override

Modified: lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java?rev=1689462&r1=1689461&r2=1689462&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java Mon Jul  6 17:16:36 2015
@@ -35,9 +35,7 @@ import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.grouping.TopGroups;
 import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.BitDocIdSet;
 import org.apache.lucene.util.BitSet;
-import org.apache.lucene.util.Bits;
 
 /**
  * This query requires that you index
@@ -83,7 +81,7 @@ import org.apache.lucene.util.Bits;
  */
 public class ToParentBlockJoinQuery extends Query {
 
-  private final BitDocIdSetFilter parentsFilter;
+  private final BitSetProducer parentsFilter;
   private final Query childQuery;
 
   // If we are rewritten, this is the original childQuery we
@@ -101,7 +99,7 @@ public class ToParentBlockJoinQuery exte
    * @param scoreMode How to aggregate multiple child scores
    * into a single parent score.
    **/
-  public ToParentBlockJoinQuery(Query childQuery, BitDocIdSetFilter parentsFilter, ScoreMode scoreMode) {
+  public ToParentBlockJoinQuery(Query childQuery, BitSetProducer parentsFilter, ScoreMode scoreMode) {
     super();
     this.origChildQuery = childQuery;
     this.childQuery = childQuery;
@@ -109,7 +107,7 @@ public class ToParentBlockJoinQuery exte
     this.scoreMode = scoreMode;
   }
 
-  private ToParentBlockJoinQuery(Query origChildQuery, Query childQuery, BitDocIdSetFilter parentsFilter, ScoreMode scoreMode) {
+  private ToParentBlockJoinQuery(Query origChildQuery, Query childQuery, BitSetProducer parentsFilter, ScoreMode scoreMode) {
     super();
     this.origChildQuery = origChildQuery;
     this.childQuery = childQuery;
@@ -130,10 +128,10 @@ public class ToParentBlockJoinQuery exte
   private static class BlockJoinWeight extends Weight {
     private final Query joinQuery;
     private final Weight childWeight;
-    private final BitDocIdSetFilter parentsFilter;
+    private final BitSetProducer parentsFilter;
     private final ScoreMode scoreMode;
 
-    public BlockJoinWeight(Query joinQuery, Weight childWeight, BitDocIdSetFilter parentsFilter, ScoreMode scoreMode) {
+    public BlockJoinWeight(Query joinQuery, Weight childWeight, BitSetProducer parentsFilter, ScoreMode scoreMode) {
       super(joinQuery);
       this.joinQuery = joinQuery;
       this.childWeight = childWeight;
@@ -173,14 +171,14 @@ public class ToParentBlockJoinQuery exte
 
       // NOTE: this does not take accept docs into account, the responsibility
       // to not match deleted docs is on the scorer
-      final BitDocIdSet parents = parentsFilter.getDocIdSet(readerContext);
+      final BitSet parents = parentsFilter.getBitSet(readerContext);
 
       if (parents == null) {
         // No matches
         return null;
       }
 
-      return new BlockJoinScorer(this, childScorer, parents.bits(), firstChildDoc, scoreMode);
+      return new BlockJoinScorer(this, childScorer, parents, firstChildDoc, scoreMode);
     }
 
     @Override

Modified: lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinSortField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinSortField.java?rev=1689462&r1=1689461&r2=1689462&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinSortField.java (original)
+++ lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinSortField.java Mon Jul  6 17:16:36 2015
@@ -25,7 +25,7 @@ import org.apache.lucene.index.SortedNum
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.search.FieldComparator;
 import org.apache.lucene.search.SortField;
-import org.apache.lucene.util.BitDocIdSet;
+import org.apache.lucene.util.BitSet;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.NumericUtils;
 
@@ -40,8 +40,8 @@ import java.io.IOException;
 public class ToParentBlockJoinSortField extends SortField {
 
   private final boolean order;
-  private final BitDocIdSetFilter parentFilter;
-  private final BitDocIdSetFilter childFilter;
+  private final BitSetProducer parentFilter;
+  private final BitSetProducer childFilter;
 
   /**
    * Create ToParentBlockJoinSortField. The parent document ordering is based on child document ordering (reverse).
@@ -52,7 +52,7 @@ public class ToParentBlockJoinSortField
    * @param parentFilter Filter that identifies the parent documents.
    * @param childFilter Filter that defines which child documents participates in sorting.
    */
-  public ToParentBlockJoinSortField(String field, Type type, boolean reverse, BitDocIdSetFilter parentFilter, BitDocIdSetFilter childFilter) {
+  public ToParentBlockJoinSortField(String field, Type type, boolean reverse, BitSetProducer parentFilter, BitSetProducer childFilter) {
     super(field, type, reverse);
     switch (getType()) {
       case STRING:
@@ -80,7 +80,7 @@ public class ToParentBlockJoinSortField
    * @param parentFilter Filter that identifies the parent documents.
    * @param childFilter Filter that defines which child documents participates in sorting.
    */
-  public ToParentBlockJoinSortField(String field, Type type, boolean reverse, boolean order, BitDocIdSetFilter parentFilter, BitDocIdSetFilter childFilter) {
+  public ToParentBlockJoinSortField(String field, Type type, boolean reverse, boolean order, BitSetProducer parentFilter, BitSetProducer childFilter) {
     super(field, type, reverse);
     this.order = order;
     this.parentFilter = parentFilter;
@@ -114,12 +114,12 @@ public class ToParentBlockJoinSortField
         final BlockJoinSelector.Type type = order
             ? BlockJoinSelector.Type.MAX
             : BlockJoinSelector.Type.MIN;
-        final BitDocIdSet parents = parentFilter.getDocIdSet(context);
-        final BitDocIdSet children = childFilter.getDocIdSet(context);
+        final BitSet parents = parentFilter.getBitSet(context);
+        final BitSet children = childFilter.getBitSet(context);
         if (children == null) {
           return DocValues.emptySorted();
         }
-        return BlockJoinSelector.wrap(sortedSet, type, parents.bits(), children.bits());
+        return BlockJoinSelector.wrap(sortedSet, type, parents, children);
       }
 
     };
@@ -133,22 +133,22 @@ public class ToParentBlockJoinSortField
         final BlockJoinSelector.Type type = order
             ? BlockJoinSelector.Type.MAX
             : BlockJoinSelector.Type.MIN;
-        final BitDocIdSet parents = parentFilter.getDocIdSet(context);
-        final BitDocIdSet children = childFilter.getDocIdSet(context);
+        final BitSet parents = parentFilter.getBitSet(context);
+        final BitSet children = childFilter.getBitSet(context);
         if (children == null) {
           return DocValues.emptyNumeric();
         }
-        return BlockJoinSelector.wrap(sortedNumeric, type, parents.bits(), children.bits());
+        return BlockJoinSelector.wrap(sortedNumeric, type, parents, children);
       }
       @Override
       protected Bits getDocsWithValue(LeafReaderContext context, String field) throws IOException {
         final Bits docsWithValue = DocValues.getDocsWithField(context.reader(), field);
-        final BitDocIdSet parents = parentFilter.getDocIdSet(context);
-        final BitDocIdSet children = childFilter.getDocIdSet(context);
+        final BitSet parents = parentFilter.getBitSet(context);
+        final BitSet children = childFilter.getBitSet(context);
         if (children == null) {
           return new Bits.MatchNoBits(context.reader().maxDoc());
         }
-        return BlockJoinSelector.wrap(docsWithValue, parents.bits(), children.bits());
+        return BlockJoinSelector.wrap(docsWithValue, parents, children);
       }
     };
   }
@@ -161,22 +161,22 @@ public class ToParentBlockJoinSortField
         final BlockJoinSelector.Type type = order
             ? BlockJoinSelector.Type.MAX
             : BlockJoinSelector.Type.MIN;
-        final BitDocIdSet parents = parentFilter.getDocIdSet(context);
-        final BitDocIdSet children = childFilter.getDocIdSet(context);
+        final BitSet parents = parentFilter.getBitSet(context);
+        final BitSet children = childFilter.getBitSet(context);
         if (children == null) {
           return DocValues.emptyNumeric();
         }
-        return BlockJoinSelector.wrap(sortedNumeric, type, parents.bits(), children.bits());
+        return BlockJoinSelector.wrap(sortedNumeric, type, parents, children);
       }
       @Override
       protected Bits getDocsWithValue(LeafReaderContext context, String field) throws IOException {
         final Bits docsWithValue = DocValues.getDocsWithField(context.reader(), field);
-        final BitDocIdSet parents = parentFilter.getDocIdSet(context);
-        final BitDocIdSet children = childFilter.getDocIdSet(context);
+        final BitSet parents = parentFilter.getBitSet(context);
+        final BitSet children = childFilter.getBitSet(context);
         if (children == null) {
           return new Bits.MatchNoBits(context.reader().maxDoc());
         }
-        return BlockJoinSelector.wrap(docsWithValue, parents.bits(), children.bits());
+        return BlockJoinSelector.wrap(docsWithValue, parents, children);
       }
     };
   }
@@ -189,12 +189,12 @@ public class ToParentBlockJoinSortField
         final BlockJoinSelector.Type type = order
             ? BlockJoinSelector.Type.MAX
             : BlockJoinSelector.Type.MIN;
-        final BitDocIdSet parents = parentFilter.getDocIdSet(context);
-        final BitDocIdSet children = childFilter.getDocIdSet(context);
+        final BitSet parents = parentFilter.getBitSet(context);
+        final BitSet children = childFilter.getBitSet(context);
         if (children == null) {
           return DocValues.emptyNumeric();
         }
-        final NumericDocValues view = BlockJoinSelector.wrap(sortedNumeric, type, parents.bits(), children.bits());
+        final NumericDocValues view = BlockJoinSelector.wrap(sortedNumeric, type, parents, children);
         // undo the numericutils sortability
         return new NumericDocValues() {
           @Override
@@ -214,12 +214,12 @@ public class ToParentBlockJoinSortField
         final BlockJoinSelector.Type type = order
             ? BlockJoinSelector.Type.MAX
             : BlockJoinSelector.Type.MIN;
-        final BitDocIdSet parents = parentFilter.getDocIdSet(context);
-        final BitDocIdSet children = childFilter.getDocIdSet(context);
+        final BitSet parents = parentFilter.getBitSet(context);
+        final BitSet children = childFilter.getBitSet(context);
         if (children == null) {
           return DocValues.emptyNumeric();
         }
-        final NumericDocValues view = BlockJoinSelector.wrap(sortedNumeric, type, parents.bits(), children.bits());
+        final NumericDocValues view = BlockJoinSelector.wrap(sortedNumeric, type, parents, children);
         // undo the numericutils sortability
         return new NumericDocValues() {
           @Override
@@ -231,12 +231,12 @@ public class ToParentBlockJoinSortField
       @Override
       protected Bits getDocsWithValue(LeafReaderContext context, String field) throws IOException {
         final Bits docsWithValue = DocValues.getDocsWithField(context.reader(), field);
-        final BitDocIdSet parents = parentFilter.getDocIdSet(context);
-        final BitDocIdSet children = childFilter.getDocIdSet(context);
+        final BitSet parents = parentFilter.getBitSet(context);
+        final BitSet children = childFilter.getBitSet(context);
         if (children == null) {
           return new Bits.MatchNoBits(context.reader().maxDoc());
         }
-        return BlockJoinSelector.wrap(docsWithValue, parents.bits(), children.bits());
+        return BlockJoinSelector.wrap(docsWithValue, parents, children);
       }
     };
   }

Modified: lucene/dev/branches/branch_5x/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java?rev=1689462&r1=1689461&r2=1689462&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java (original)
+++ lucene/dev/branches/branch_5x/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java Mon Jul  6 17:16:36 2015
@@ -139,7 +139,7 @@ public class TestBlockJoin extends Lucen
     w.close();
     assertTrue(r.leaves().size() > 1);
     IndexSearcher s = new IndexSearcher(r);
-    BitDocIdSetFilter parentsFilter = new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("docType", "resume"))));
+    BitSetProducer parentsFilter = new QueryBitSetProducer(new TermQuery(new Term("docType", "resume")));
 
     BooleanQuery childQuery = new BooleanQuery();
     childQuery.add(new BooleanClause(new TermQuery(new Term("skill", "java")), Occur.MUST));
@@ -191,7 +191,7 @@ public class TestBlockJoin extends Lucen
     IndexSearcher s = newSearcher(r);
 
     // Create a filter that defines "parent" documents in the index - in this case resumes
-    BitDocIdSetFilter parentsFilter = new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("docType", "resume"))));
+    BitSetProducer parentsFilter = new QueryBitSetProducer(new TermQuery(new Term("docType", "resume")));
 
     // Define child document criteria (finds an example of relevant work experience)
     BooleanQuery childQuery = new BooleanQuery();
@@ -282,7 +282,7 @@ public class TestBlockJoin extends Lucen
     // iterations: 
     qc.setRewriteMethod(MultiTermQuery.CONSTANT_SCORE_BOOLEAN_REWRITE);
 
-    BitDocIdSetFilter parentsFilter = new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("docType", "resume"))));
+    BitSetProducer parentsFilter = new QueryBitSetProducer(new TermQuery(new Term("docType", "resume")));
 
     int h1 = qc.hashCode();
     Query qw1 = qc.rewrite(r);
@@ -307,7 +307,7 @@ public class TestBlockJoin extends Lucen
     dir.close();
   }
 
-  protected QueryWrapperFilter skill(String skill) {
+  protected Filter skill(String skill) {
     return new QueryWrapperFilter(new TermQuery(new Term("skill", skill)));
   }
 
@@ -359,12 +359,13 @@ public class TestBlockJoin extends Lucen
       
     assertEquals("no filter - both passed", 2, s.search(childJoinQuery, 10).totalHits);
 
-    assertEquals("dummy filter passes everyone ", 2, s.search(new FilteredQuery(childJoinQuery, parentsFilter), 10).totalHits);
-    assertEquals("dummy filter passes everyone ", 2, s.search(new FilteredQuery(childJoinQuery, new QueryWrapperFilter(new TermQuery(new Term("docType", "resume")))), 10).totalHits);
+    BooleanQuery query = new BooleanQuery();
+    query.add(childJoinQuery, Occur.MUST);
+    query.add(new TermQuery(new Term("docType", "resume")), Occur.FILTER);
+    assertEquals("dummy filter passes everyone ", 2, s.search(query, 10).totalHits);
       
     // not found test
     assertEquals("noone live there", 0, s.search(new FilteredQuery(childJoinQuery, new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("country", "Oz"))))), 1).totalHits);
-    assertEquals("noone live there", 0, s.search(new FilteredQuery(childJoinQuery, new QueryWrapperFilter(new TermQuery(new Term("country", "Oz")))), 1).totalHits);
       
     // apply the UK filter by the searcher
     TopDocs ukOnly = s.search(new FilteredQuery(childJoinQuery, new QueryWrapperFilter(parentQuery)), 1);
@@ -401,11 +402,11 @@ public class TestBlockJoin extends Lucen
     }
   }
   
-  private Document getParentDoc(IndexReader reader, BitDocIdSetFilter parents, int childDocID) throws IOException {
+  private Document getParentDoc(IndexReader reader, BitSetProducer parents, int childDocID) throws IOException {
     final List<LeafReaderContext> leaves = reader.leaves();
     final int subIndex = ReaderUtil.subIndex(childDocID, leaves);
     final LeafReaderContext leaf = leaves.get(subIndex);
-    final BitSet bits = parents.getDocIdSet(leaf).bits();
+    final BitSet bits = parents.getBitSet(leaf);
     return leaf.reader().document(bits.nextSetBit(childDocID - leaf.docBase));
   }
   
@@ -416,7 +417,7 @@ public class TestBlockJoin extends Lucen
     w.close();
     IndexSearcher s = newSearcher(r);
     
-    ToParentBlockJoinQuery q = new ToParentBlockJoinQuery(new MatchNoDocsQuery(), new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(new MatchAllDocsQuery())), ScoreMode.Avg);
+    ToParentBlockJoinQuery q = new ToParentBlockJoinQuery(new MatchNoDocsQuery(), new QueryBitSetProducer(new MatchAllDocsQuery()), ScoreMode.Avg);
     QueryUtils.check(random(), q, s);
     s.search(q, 10);
     BooleanQuery bq = new BooleanQuery();
@@ -626,7 +627,7 @@ public class TestBlockJoin extends Lucen
 
     final IndexSearcher joinS = new IndexSearcher(joinR);
 
-    final BitDocIdSetFilter parentsFilter = new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("isParent", "x"))));
+    final BitSetProducer parentsFilter = new QueryBitSetProducer(new TermQuery(new Term("isParent", "x")));
 
     final int iters = 200*RANDOM_MULTIPLIER;
 
@@ -874,9 +875,9 @@ public class TestBlockJoin extends Lucen
         childJoinQuery2 = parentJoinQuery2;
       } else {
         final Term childTerm = randomChildTerm(childFields[0]);
+        final Filter f = new QueryWrapperFilter(new TermQuery(childTerm));
         if (random().nextBoolean()) { // filtered case
           childJoinQuery2 = parentJoinQuery2;
-          final Filter f = new QueryWrapperFilter(new TermQuery(childTerm));
           childJoinQuery2 = new FilteredQuery(childJoinQuery2, random().nextBoolean()
                   ? new BitDocIdSetCachingWrapperFilter(f): f);
         } else {
@@ -896,7 +897,6 @@ public class TestBlockJoin extends Lucen
         
         if (random().nextBoolean()) { // filtered case
           childQuery2 = parentQuery2;
-          final Filter f = new QueryWrapperFilter(new TermQuery(childTerm));
           childQuery2 = new FilteredQuery(childQuery2, random().nextBoolean()
                   ? new BitDocIdSetCachingWrapperFilter(f): f);
         } else {
@@ -1033,7 +1033,7 @@ public class TestBlockJoin extends Lucen
     IndexSearcher s = newSearcher(r);
 
     // Create a filter that defines "parent" documents in the index - in this case resumes
-    BitDocIdSetFilter parentsFilter = new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("docType", "resume"))));
+    BitSetProducer parentsFilter = new QueryBitSetProducer(new TermQuery(new Term("docType", "resume")));
 
     // Define child document criteria (finds an example of relevant work experience)
     BooleanQuery childJobQuery = new BooleanQuery();
@@ -1113,9 +1113,8 @@ public class TestBlockJoin extends Lucen
     w.close();
     IndexSearcher s = newSearcher(r);
     Query tq = new TermQuery(new Term("child", "1"));
-    BitDocIdSetFilter parentFilter = new BitDocIdSetCachingWrapperFilter(
-                            new QueryWrapperFilter(
-                              new TermQuery(new Term("parent", "1"))));
+    BitSetProducer parentFilter = new QueryBitSetProducer(
+                              new TermQuery(new Term("parent", "1")));
 
     ToParentBlockJoinQuery q = new ToParentBlockJoinQuery(tq, parentFilter, ScoreMode.Avg);
     Weight weight = s.createNormalizedWeight(q, true);
@@ -1147,9 +1146,8 @@ public class TestBlockJoin extends Lucen
     w.close();
     IndexSearcher s = newSearcher(r);
     Query tq = new TermQuery(new Term("child", "2"));
-    BitDocIdSetFilter parentFilter = new BitDocIdSetCachingWrapperFilter(
-                            new QueryWrapperFilter(
-                              new TermQuery(new Term("isparent", "yes"))));
+    BitSetProducer parentFilter = new QueryBitSetProducer(
+                              new TermQuery(new Term("isparent", "yes")));
 
     ToParentBlockJoinQuery q = new ToParentBlockJoinQuery(tq, parentFilter, ScoreMode.Avg);
     Weight weight = s.createNormalizedWeight(q, true);
@@ -1181,7 +1179,7 @@ public class TestBlockJoin extends Lucen
     IndexSearcher s = new IndexSearcher(r);
 
     // Create a filter that defines "parent" documents in the index - in this case resumes
-    BitDocIdSetFilter parentsFilter = new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("docType", "resume"))));
+    BitSetProducer parentsFilter = new QueryBitSetProducer(new TermQuery(new Term("docType", "resume")));
 
     // Define child document criteria (finds an example of relevant work experience)
     BooleanQuery childQuery = new BooleanQuery();
@@ -1287,7 +1285,7 @@ public class TestBlockJoin extends Lucen
 
     IndexSearcher searcher = new ToParentBlockJoinIndexSearcher(r);
     Query childQuery = new TermQuery(new Term("childText", "text"));
-    BitDocIdSetFilter parentsFilter = new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("isParent", "yes"))));
+    BitSetProducer parentsFilter = new QueryBitSetProducer(new TermQuery(new Term("isParent", "yes")));
     ToParentBlockJoinQuery childJoinQuery = new ToParentBlockJoinQuery(childQuery, parentsFilter, ScoreMode.Avg);
     BooleanQuery parentQuery = new BooleanQuery();
     parentQuery.add(childJoinQuery, Occur.SHOULD);
@@ -1357,7 +1355,7 @@ public class TestBlockJoin extends Lucen
     
     // never matches:
     Query childQuery = new TermQuery(new Term("childText", "bogus"));
-    BitDocIdSetFilter parentsFilter = new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("isParent", "yes"))));
+    BitSetProducer parentsFilter = new QueryBitSetProducer(new TermQuery(new Term("isParent", "yes")));
     ToParentBlockJoinQuery childJoinQuery = new ToParentBlockJoinQuery(childQuery, parentsFilter, ScoreMode.Avg);
     BooleanQuery parentQuery = new BooleanQuery();
     parentQuery.add(childJoinQuery, Occur.SHOULD);
@@ -1422,7 +1420,7 @@ public class TestBlockJoin extends Lucen
 
     // illegally matches parent:
     Query childQuery = new TermQuery(new Term("parentText", "text"));
-    BitDocIdSetFilter parentsFilter = new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("isParent", "yes"))));
+    BitSetProducer parentsFilter = new QueryBitSetProducer(new TermQuery(new Term("isParent", "yes")));
     ToParentBlockJoinQuery childJoinQuery = new ToParentBlockJoinQuery(childQuery, parentsFilter, ScoreMode.Avg);
     BooleanQuery parentQuery = new BooleanQuery();
     parentQuery.add(childJoinQuery, Occur.SHOULD);
@@ -1474,7 +1472,7 @@ public class TestBlockJoin extends Lucen
     IndexSearcher s = newSearcher(r);
 
     // Create a filter that defines "parent" documents in the index - in this case resumes
-    BitDocIdSetFilter parentsFilter = new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("isparent", "yes"))));
+    BitSetProducer parentsFilter = new QueryBitSetProducer(new TermQuery(new Term("isparent", "yes")));
 
     Query parentQuery = new TermQuery(new Term("parent", "2"));
 
@@ -1509,7 +1507,7 @@ public class TestBlockJoin extends Lucen
     final IndexSearcher searcher = newSearcher(reader);
     searcher.setQueryCache(null); // to have real advance() calls
 
-    final BitDocIdSetFilter parentsFilter = new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("parent", "true"))));
+    final BitSetProducer parentsFilter = new QueryBitSetProducer(new TermQuery(new Term("parent", "true")));
     final Query toChild = new ToChildBlockJoinQuery(new TermQuery(new Term("foo_parent", "bar")), parentsFilter);
     final Query childQuery = new TermQuery(new Term("foo_child", "baz"));
 
@@ -1551,7 +1549,7 @@ public class TestBlockJoin extends Lucen
     IndexSearcher s = newSearcher(r);
 
     // Create a filter that defines "parent" documents in the index - in this case resumes
-    BitDocIdSetFilter parentsFilter = new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("docType", "resume"))));
+    BitSetProducer parentsFilter = new QueryBitSetProducer(new TermQuery(new Term("docType", "resume")));
     Query parentQuery = new PrefixQuery(new Term("country", "United"));
     
     ToChildBlockJoinQuery toChildQuery = new ToChildBlockJoinQuery(parentQuery, parentsFilter);
@@ -1589,7 +1587,7 @@ public class TestBlockJoin extends Lucen
     IndexSearcher s = newSearcher(r);
 
     // Create a filter that defines "parent" documents in the index - in this case resumes
-    BitDocIdSetFilter parentsFilter = new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("docType", "resume"))));
+    BitSetProducer parentsFilter = new QueryBitSetProducer(new TermQuery(new Term("docType", "resume")));
     Query parentQuery = new PrefixQuery(new Term("country", "United"));
     
     ToChildBlockJoinQuery toChildQuery = new ToChildBlockJoinQuery(parentQuery, parentsFilter);

Modified: lucene/dev/branches/branch_5x/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinSorting.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinSorting.java?rev=1689462&r1=1689461&r2=1689462&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinSorting.java (original)
+++ lucene/dev/branches/branch_5x/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinSorting.java Mon Jul  6 17:16:36 2015
@@ -27,12 +27,8 @@ import org.apache.lucene.index.NoMergePo
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.FieldDoc;
-import org.apache.lucene.search.Filter;
-import org.apache.lucene.search.FilteredQuery;
 import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.PrefixQuery;
-import org.apache.lucene.search.QueryWrapperFilter;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.TermQuery;
@@ -231,11 +227,11 @@ public class TestBlockJoinSorting extend
 
     IndexSearcher searcher = new IndexSearcher(DirectoryReader.open(w.w, false));
     w.close();
-    BitDocIdSetFilter parentFilter = new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("__type", "parent"))));
-    BitDocIdSetFilter childFilter = new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(new PrefixQuery(new Term("field2"))));
+    BitSetProducer parentFilter = new QueryBitSetProducer(new TermQuery(new Term("__type", "parent")));
+    BitSetProducer childFilter = new QueryBitSetProducer(new PrefixQuery(new Term("field2")));
     ToParentBlockJoinQuery query = new ToParentBlockJoinQuery(
-        new FilteredQuery(new MatchAllDocsQuery(), childFilter),
-        new BitDocIdSetCachingWrapperFilter(parentFilter),
+        new PrefixQuery(new Term("field2")),
+        parentFilter,
         ScoreMode.None
     );
 
@@ -297,10 +293,10 @@ public class TestBlockJoinSorting extend
     assertEquals("g", ((BytesRef) ((FieldDoc) topDocs.scoreDocs[4]).fields[0]).utf8ToString());
 
     // Sort by field descending, order last, sort filter (filter_1:T)
-    childFilter = new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(new TermQuery((new Term("filter_1", "T")))));
+    childFilter = new QueryBitSetProducer(new TermQuery((new Term("filter_1", "T"))));
     query = new ToParentBlockJoinQuery(
-        new FilteredQuery(new MatchAllDocsQuery(), childFilter),
-        new BitDocIdSetCachingWrapperFilter(parentFilter),
+        new TermQuery((new Term("filter_1", "T"))),
+        parentFilter,
         ScoreMode.None
     );
     sortField = new ToParentBlockJoinSortField(

Modified: lucene/dev/branches/branch_5x/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinValidation.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinValidation.java?rev=1689462&r1=1689461&r2=1689462&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinValidation.java (original)
+++ lucene/dev/branches/branch_5x/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinValidation.java Mon Jul  6 17:16:36 2015
@@ -34,7 +34,6 @@ import org.apache.lucene.search.BooleanQ
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.search.QueryWrapperFilter;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.Weight;
@@ -54,7 +53,7 @@ public class TestBlockJoinValidation ext
   private Directory directory;
   private IndexReader indexReader;
   private IndexSearcher indexSearcher;
-  private BitDocIdSetFilter parentsFilter;
+  private BitSetProducer parentsFilter;
 
   @Override
   public void setUp() throws Exception {
@@ -70,7 +69,7 @@ public class TestBlockJoinValidation ext
     indexReader = DirectoryReader.open(indexWriter, random().nextBoolean());
     indexWriter.close();
     indexSearcher = new IndexSearcher(indexReader);
-    parentsFilter = new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(new WildcardQuery(new Term("parent", "*"))));
+    parentsFilter = new QueryBitSetProducer(new WildcardQuery(new Term("parent", "*")));
   }
 
   @Override
@@ -132,7 +131,7 @@ public class TestBlockJoinValidation ext
     final LeafReaderContext context = indexSearcher.getIndexReader().leaves().get(0);
     Weight weight = indexSearcher.createNormalizedWeight(blockJoinQuery, true);
     Scorer scorer = weight.scorer(context);
-    final Bits parentDocs = parentsFilter.getDocIdSet(context).bits();
+    final Bits parentDocs = parentsFilter.getBitSet(context);
 
     int target;
     do {

Modified: lucene/dev/branches/branch_5x/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestHierarchicalDocBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestHierarchicalDocBuilder.java?rev=1689462&r1=1689461&r2=1689462&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestHierarchicalDocBuilder.java (original)
+++ lucene/dev/branches/branch_5x/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestHierarchicalDocBuilder.java Mon Jul  6 17:16:36 2015
@@ -34,8 +34,8 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.QueryWrapperFilter;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
-import org.apache.lucene.search.join.BitDocIdSetCachingWrapperFilter;
-import org.apache.lucene.search.join.BitDocIdSetFilter;
+import org.apache.lucene.search.join.QueryBitSetProducer;
+import org.apache.lucene.search.join.BitSetProducer;
 import org.apache.lucene.search.join.ScoreMode;
 import org.apache.lucene.search.join.ToParentBlockJoinQuery;
 import org.apache.solr.common.util.StrUtils;
@@ -457,10 +457,10 @@ public class TestHierarchicalDocBuilder
    **/
   private final String childEntityTemplate = "<entity " + ConfigNameConstants.CHILD + "=\"true\" name=\"{0}\" query=\"{1}\">\n {2} {3} </entity>\n";
   
-  private BitDocIdSetFilter createParentFilter(String type) {
+  private BitSetProducer createParentFilter(String type) {
     BooleanQuery parentQuery = new BooleanQuery();
     parentQuery.add(new TermQuery(new Term("type_s", type)), Occur.MUST);
-    return new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(parentQuery));
+    return new QueryBitSetProducer(new QueryWrapperFilter(parentQuery));
   }
   
   private String nextId() {

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/response/transform/ChildDocTransformerFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/response/transform/ChildDocTransformerFactory.java?rev=1689462&r1=1689461&r2=1689462&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/response/transform/ChildDocTransformerFactory.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/response/transform/ChildDocTransformerFactory.java Mon Jul  6 17:16:36 2015
@@ -24,8 +24,8 @@ import org.apache.lucene.index.Indexable
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.QueryWrapperFilter;
 import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.join.BitDocIdSetCachingWrapperFilter;
-import org.apache.lucene.search.join.BitDocIdSetFilter;
+import org.apache.lucene.search.join.QueryBitSetProducer;
+import org.apache.lucene.search.join.BitSetProducer;
 import org.apache.lucene.search.join.ToChildBlockJoinQuery;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrException;
@@ -76,10 +76,10 @@ public class ChildDocTransformerFactory
     String childFilter = params.get( "childFilter" );
     int limit = params.getInt( "limit", 10 );
 
-    BitDocIdSetFilter parentsFilter = null;
+    BitSetProducer parentsFilter = null;
     try {
       Query parentFilterQuery = QParser.getParser( parentFilter, null, req).getQuery();
-      parentsFilter = new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(parentFilterQuery));
+      parentsFilter = new QueryBitSetProducer(new QueryWrapperFilter(parentFilterQuery));
     } catch (SyntaxError syntaxError) {
       throw new SolrException( ErrorCode.BAD_REQUEST, "Failed to create correct parent filter query" );
     }
@@ -101,11 +101,11 @@ class ChildDocTransformer extends Transf
   private final String name;
   private final SchemaField idField;
   private final IndexSchema schema;
-  private BitDocIdSetFilter parentsFilter;
+  private BitSetProducer parentsFilter;
   private Query childFilterQuery;
   private int limit;
 
-  public ChildDocTransformer( String name, final BitDocIdSetFilter parentsFilter, 
+  public ChildDocTransformer( String name, final BitSetProducer parentsFilter, 
                               final SchemaField idField, IndexSchema schema,
                               final Query childFilterQuery, int limit) {
     this.name = name;

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/join/BlockJoinChildQParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/join/BlockJoinChildQParser.java?rev=1689462&r1=1689461&r2=1689462&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/join/BlockJoinChildQParser.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/join/BlockJoinChildQParser.java Mon Jul  6 17:16:36 2015
@@ -29,7 +29,7 @@ public class BlockJoinChildQParser exten
   }
 
   protected Query createQuery(Query parentListQuery, Query query) {
-    return new ToChildBlockJoinQuery(query, getFilter(parentListQuery));
+    return new ToChildBlockJoinQuery(query, getFilter(parentListQuery).filter);
   }
 
   @Override

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/join/BlockJoinParentQParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/join/BlockJoinParentQParser.java?rev=1689462&r1=1689461&r2=1689462&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/join/BlockJoinParentQParser.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/join/BlockJoinParentQParser.java Mon Jul  6 17:16:36 2015
@@ -17,13 +17,21 @@
 
 package org.apache.solr.search.join;
 
+import java.io.IOException;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.BitsFilteredDocIdSet;
+import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.Filter;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.QueryWrapperFilter;
-import org.apache.lucene.search.join.BitDocIdSetCachingWrapperFilter;
-import org.apache.lucene.search.join.BitDocIdSetFilter;
+import org.apache.lucene.search.join.QueryBitSetProducer;
+import org.apache.lucene.search.join.BitSetProducer;
 import org.apache.lucene.search.join.ScoreMode;
 import org.apache.lucene.search.join.ToParentBlockJoinQuery;
+import org.apache.lucene.util.BitDocIdSet;
+import org.apache.lucene.util.BitSet;
+import org.apache.lucene.util.Bits;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.search.QParser;
@@ -63,21 +71,21 @@ class BlockJoinParentQParser extends QPa
   }
 
   protected Query createQuery(Query parentList, Query query) {
-    return new ToParentBlockJoinQuery(query, getFilter(parentList), ScoreMode.None);
+    return new ToParentBlockJoinQuery(query, getFilter(parentList).filter, ScoreMode.None);
   }
 
-  BitDocIdSetFilter getFilter(Query parentList) {
+  BitDocIdSetFilterWrapper getFilter(Query parentList) {
     SolrCache parentCache = req.getSearcher().getCache(CACHE_NAME);
     // lazily retrieve from solr cache
     Filter filter = null;
     if (parentCache != null) {
       filter = (Filter) parentCache.get(parentList);
     }
-    BitDocIdSetFilter result;
-    if (filter instanceof BitDocIdSetFilter) {
-      result = (BitDocIdSetFilter) filter;
+    BitDocIdSetFilterWrapper result;
+    if (filter instanceof BitDocIdSetFilterWrapper) {
+      result = (BitDocIdSetFilterWrapper) filter;
     } else {
-      result = createParentFilter(parentList);
+      result = new BitDocIdSetFilterWrapper(createParentFilter(parentList));
       if (parentCache != null) {
         parentCache.put(parentList, result);
       }
@@ -85,9 +93,48 @@ class BlockJoinParentQParser extends QPa
     return result;
   }
 
-  private BitDocIdSetFilter createParentFilter(Query parentQ) {
-    return new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(parentQ));
+  private BitSetProducer createParentFilter(Query parentQ) {
+    return new QueryBitSetProducer(new QueryWrapperFilter(parentQ));
+  }
+
+  // We need this wrapper since BitDocIdSetFilter does not extend Filter
+  static class BitDocIdSetFilterWrapper extends Filter {
+
+    final BitSetProducer filter;
+
+    BitDocIdSetFilterWrapper(BitSetProducer filter) {
+      this.filter = filter;
+    }
+
+    @Override
+    public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
+      BitSet set = filter.getBitSet(context);
+      if (set == null) {
+        return null;
+      }
+      return BitsFilteredDocIdSet.wrap(new BitDocIdSet(set), acceptDocs);
+    }
+
+    @Override
+    public String toString(String field) {
+      return getClass().getSimpleName() + "(" + filter + ")";
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (super.equals(obj) == false) {
+        return false;
+      }
+      return filter.equals(((BitDocIdSetFilterWrapper) obj).filter);
+    }
+
+    @Override
+    public int hashCode() {
+      return 31 * super.hashCode() + filter.hashCode();
+    }
+
   }
+
 }
 
 

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/update/AddBlockUpdateTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/update/AddBlockUpdateTest.java?rev=1689462&r1=1689461&r2=1689462&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/update/AddBlockUpdateTest.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/update/AddBlockUpdateTest.java Mon Jul  6 17:16:36 2015
@@ -26,7 +26,7 @@ import org.apache.lucene.search.QueryWra
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TermRangeQuery;
 import org.apache.lucene.search.TopDocs;
-import org.apache.lucene.search.join.BitDocIdSetCachingWrapperFilter;
+import org.apache.lucene.search.join.QueryBitSetProducer;
 import org.apache.lucene.search.join.ScoreMode;
 import org.apache.lucene.search.join.ToParentBlockJoinQuery;
 import org.apache.solr.SolrTestCaseJ4;
@@ -562,7 +562,7 @@ public class AddBlockUpdateTest extends
   
   protected ToParentBlockJoinQuery join(final String childTerm) {
     return new ToParentBlockJoinQuery(
-        new TermQuery(new Term(child, childTerm)), new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(
+        new TermQuery(new Term(child, childTerm)), new QueryBitSetProducer(new QueryWrapperFilter(
             new TermRangeQuery(parent, null, null, false, false))), ScoreMode.None);
   }