You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mk...@apache.org on 2019/12/18 08:17:22 UTC

[lucene-solr] branch master updated: SOLR-14051: removing Block Join Facet Component in favor of uniqueBlock(\_root_).

This is an automated email from the ASF dual-hosted git repository.

mkhl pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/master by this push:
     new b0e532b  SOLR-14051: removing Block Join Facet Component in favor of uniqueBlock(\_root_).
b0e532b is described below

commit b0e532b7f32c1822e93325ddcf43d772c7ae8073
Author: Mikhail Khludnev <mk...@apache.org>
AuthorDate: Sun Dec 15 00:34:52 2019 +0300

    SOLR-14051: removing Block Join Facet Component in favor of uniqueBlock(\_root_).
---
 solr/CHANGES.txt                                   |   6 +
 .../org/apache/solr/search/join/BitSetSlice.java   |  45 --
 .../search/join/BlockJoinDocSetFacetComponent.java | 195 -------
 .../solr/search/join/BlockJoinFacetAccsHolder.java |  83 ---
 .../solr/search/join/BlockJoinFacetComponent.java  |  23 -
 .../join/BlockJoinFacetComponentSupport.java       | 158 -----
 .../solr/search/join/BlockJoinFacetFilter.java     |  96 ----
 .../join/BlockJoinFieldFacetAccumulator.java       | 235 --------
 .../solr/search/join/BlockJoinParentQParser.java   |   1 +
 .../search/join/ChildFieldValueSourceParser.java   |   7 +-
 .../apache/solr/search/join/ScoreModeParser.java   |   1 +
 .../conf/schema-blockjoinfacetcomponent.xml        |  40 --
 .../conf/solrconfig-blockjoinfacetcomponent.xml    |  58 --
 .../org/apache/solr/search/join/BJQParserTest.java |   6 +-
 .../search/join/BlockJoinFacetDistribTest.java     | 237 --------
 .../solr/search/join/BlockJoinFacetRandomTest.java | 637 ---------------------
 .../solr/search/join/BlockJoinFacetSimpleTest.java | 121 ----
 .../solr/search/join/TestScoreJoinQPNoScore.java   |   5 +-
 .../solr/search/join/TestScoreJoinQPScore.java     |   4 +-
 .../join/another/BJQFilterAccessibleTest.java      |   3 +-
 solr/solr-ref-guide/src/blockjoin-faceting.adoc    | 115 ----
 solr/solr-ref-guide/src/faceting.adoc              |   1 -
 .../src/json-faceting-domain-changes.adoc          |   2 +-
 .../src/searching-nested-documents.adoc            |   2 +-
 24 files changed, 28 insertions(+), 2053 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 31d926b..4d26026 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -75,6 +75,9 @@ Upgrade Notes
 * SOLR-13817: Legacy SolrCache implementations (LRUCache, LFUCache, FastLRUCache) have been removed.
   Users have to modify their existing configurations to use CaffeineCache instead. (ab)
 
+* SOLR-14092: Deprecated BlockJoinFacetComponent and BlockJoinDocSetFacetComponent are removed
+  Users are encouraged to migrate to uniqueBlock() in JSON Facet API.  (Mikhail Khludnev)
+  
 Improvements
 ----------------------
 
@@ -110,6 +113,9 @@ Upgrade Notes
 
 * SOLR-14026: Upgrade Jetty to 9.4.24.v20191120 and dropwizard to 4.1.2 (Erick Erickson)
 
+* SOLR-14092: BlockJoinFacetComponent is marked for deprecation and will be removed in 9.0.
+  Users are encouraged to migrate to uniqueBlock() in JSON Facet API.  (Mikhail Khludnev)
+
 New Features
 ---------------------
 (No changes)
diff --git a/solr/core/src/java/org/apache/solr/search/join/BitSetSlice.java b/solr/core/src/java/org/apache/solr/search/join/BitSetSlice.java
deleted file mode 100644
index 824c4e2..0000000
--- a/solr/core/src/java/org/apache/solr/search/join/BitSetSlice.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.search.join;
-
-import org.apache.lucene.util.FixedBitSet;
-
-class BitSetSlice {
-  private final FixedBitSet fbs;
-  private final int off;
-  private final int len;
-
-  BitSetSlice(FixedBitSet fbs, int off, int len) {
-    this.fbs = fbs;
-    this.off = off;
-    this.len = len;
-  }
-
-  public boolean get(int pos) {
-    return fbs.get(pos + off);
-  }
-
-  public int prevSetBit(int pos) {
-    int result = fbs.prevSetBit(pos + off) - off;
-    return (result < 0) ? -1 : result;
-  }
-
-  public int nextSetBit(int pos) {
-    int result = fbs.nextSetBit(pos + off) - off;
-    return (result >= len) ? -1 : result;
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/search/join/BlockJoinDocSetFacetComponent.java b/solr/core/src/java/org/apache/solr/search/join/BlockJoinDocSetFacetComponent.java
deleted file mode 100644
index b8f3034..0000000
--- a/solr/core/src/java/org/apache/solr/search/join/BlockJoinDocSetFacetComponent.java
+++ /dev/null
@@ -1,195 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.search.join;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.search.BooleanClause;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.DocIdSet;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.MatchAllDocsQuery;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.join.ToParentBlockJoinQuery;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.handler.component.ResponseBuilder;
-import org.apache.solr.search.BitDocSet;
-import org.apache.solr.search.DocSet;
-import org.apache.solr.search.Filter;
-import org.apache.solr.search.QueryContext;
-import org.apache.solr.search.facet.BlockJoin;
-import org.apache.solr.search.join.BlockJoinFieldFacetAccumulator.AggregatableDocIter;
-
-/**
- * Calculates facets on children documents and aggregates hits by parent documents.
- * Enables when child.facet.field parameter specifies a field name for faceting. 
- * So far it supports string fields only. It requires to search by {@link ToParentBlockJoinQuery}.
- * */
-public class BlockJoinDocSetFacetComponent extends BlockJoinFacetComponentSupport {
-  
-  private final String bjqKey = this.getClass().getSimpleName()+".bjq";
-  
-  private static final class SegmentChildren implements AggregatableDocIter {
-    
-    private final BitDocSet allParentsBitsDocSet;
-    private int nextDoc = DocIdSetIterator.NO_MORE_DOCS;
-    private DocIdSetIterator disi;
-    private int currentParent=-1;
-    final LeafReaderContext segment;
-    final DocIdSet childrenMatches;
-    
-    private SegmentChildren(LeafReaderContext subCtx, DocIdSet dis, BitDocSet allParentsBitsDocSet) {
-      this.allParentsBitsDocSet = allParentsBitsDocSet;
-      this.childrenMatches = dis;
-      this.segment = subCtx;
-      reset();
-    }
-    
-    @Override
-    public Integer next() {
-      return nextDoc();
-    }
-    
-    @Override
-    public boolean hasNext() {
-      return nextDoc != DocIdSetIterator.NO_MORE_DOCS;
-    }
-    
-    @Override
-    public float score() {
-      return 0;
-    }
-    
-    @Override
-    public int nextDoc() {
-      int lastDoc = nextDoc;
-      assert nextDoc != DocIdSetIterator.NO_MORE_DOCS;
-      if (lastDoc>currentParent) { // we passed the previous block, and need to reevaluate a parent
-        currentParent = allParentsBitsDocSet.getBits().nextSetBit(lastDoc+segment.docBase)-segment.docBase;
-      }
-      try {
-        nextDoc = disi.nextDoc();
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-      return lastDoc;
-    }
-    
-    @Override
-    public void reset() {
-      currentParent=-1;
-      try {
-        disi = childrenMatches.iterator();
-        if (disi != null) {
-          nextDoc = disi.nextDoc();
-        }else{
-          nextDoc = DocIdSetIterator.NO_MORE_DOCS;
-        }
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    }
-    
-    @Override
-    public int getAggKey() {
-      return currentParent;
-    }
-  }
-
-  public BlockJoinDocSetFacetComponent() {}
-  
-  @Override
-  public void prepare(ResponseBuilder rb) throws IOException {
-    
-    if (getChildFacetFields(rb.req) != null) {
-      validateQuery(rb.getQuery());
-      rb.setNeedDocSet(true);
-      rb.req.getContext().put(bjqKey, extractChildQuery(rb.getQuery()));
-    }
-  }
-  
-  private ToParentBlockJoinQuery extractChildQuery(Query query) {
-    if (!(query instanceof ToParentBlockJoinQuery)) {
-      if (query instanceof BooleanQuery) {
-        List<BooleanClause> clauses = ((BooleanQuery) query).clauses();
-        ToParentBlockJoinQuery once = null;
-        for (BooleanClause clause : clauses) {
-          if (clause.getQuery() instanceof ToParentBlockJoinQuery) {
-            if (once==null) {
-              once = (ToParentBlockJoinQuery) clause.getQuery(); 
-            } else {
-              throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "can't choose between " +
-                         once + " and " + clause.getQuery());
-            }
-          }
-        }
-        if (once!=null) {
-          return once;
-        }
-      }
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NO_TO_PARENT_BJQ_MESSAGE);
-    }
-    else{
-      return (ToParentBlockJoinQuery) query;
-    }
-  }
-
-  @Override
-  public void process(ResponseBuilder rb) throws IOException {
-    final BlockJoinParentQParser.AllParentsAware bjq = 
-        (BlockJoinParentQParser.AllParentsAware) rb.req.getContext().get(bjqKey);
-    if(bjq!=null){
-      final DocSet parentResult = rb.getResults().docSet;
-      final BitDocSet allParentsBitsDocSet = rb.req.getSearcher().getDocSetBits(bjq.getParentQuery());
-      final DocSet allChildren = BlockJoin.toChildren(parentResult, 
-          allParentsBitsDocSet,
-          rb.req.getSearcher().getDocSetBits( new MatchAllDocsQuery() ), 
-          QueryContext.newContext(rb.req.getSearcher()));
-      
-      final DocSet childQueryDocSet = rb.req.getSearcher().getDocSet(bjq.getChildQuery());
-      final DocSet selectedChildren = allChildren.intersection(childQueryDocSet);
-      
-      // don't include parent into facet counts
-      //childResult = childResult.union(parentResult);// just to mimic the current logic
-      
-      final List<LeafReaderContext> leaves = rb.req.getSearcher().getIndexReader().leaves();
-      
-      Filter filter = selectedChildren.getTopFilter();
-
-      final BlockJoinFacetAccsHolder facetCounter = new BlockJoinFacetAccsHolder(rb.req);
-      
-      for (int subIdx = 0; subIdx < leaves.size(); subIdx++) {
-        LeafReaderContext subCtx = leaves.get(subIdx);
-        DocIdSet dis = filter.getDocIdSet(subCtx, null); // solr docsets already exclude any deleted docs
-        
-        AggregatableDocIter iter = new SegmentChildren(subCtx, dis, allParentsBitsDocSet);
-        
-        if (iter.hasNext()){
-          facetCounter.doSetNextReader(subCtx);
-          facetCounter.countFacets(iter);
-        }
-      }
-      facetCounter.finish();
-      
-      rb.req.getContext().put(COLLECTOR_CONTEXT_PARAM,facetCounter);
-      super.process(rb);
-    }
-    
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/search/join/BlockJoinFacetAccsHolder.java b/solr/core/src/java/org/apache/solr/search/join/BlockJoinFacetAccsHolder.java
deleted file mode 100644
index 769a9c8..0000000
--- a/solr/core/src/java/org/apache/solr/search/join/BlockJoinFacetAccsHolder.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.search.join;
-
-import java.io.IOException;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.search.join.BlockJoinFieldFacetAccumulator.AggregatableDocIter;
-
-/**
- * For each collected parent document creates matched block, which is a docSet with matched children and parent doc
- * itself. Then updates each BlockJoinFieldFacetAccumulator with the created matched block.
- */
-class BlockJoinFacetAccsHolder {
-  private BlockJoinFieldFacetAccumulator[] blockJoinFieldFacetAccumulators;
-  private boolean firstSegment = true;
-  
-  BlockJoinFacetAccsHolder(SolrQueryRequest req) throws IOException {
-    String[] facetFieldNames = BlockJoinFacetComponentSupport.getChildFacetFields(req);
-    assert facetFieldNames != null;
-    blockJoinFieldFacetAccumulators = new BlockJoinFieldFacetAccumulator[facetFieldNames.length];
-    for (int i = 0; i < facetFieldNames.length; i++) {
-      blockJoinFieldFacetAccumulators[i] = new BlockJoinFieldFacetAccumulator(facetFieldNames[i], req.getSearcher());
-    }
-  }
-  
-
-  protected void doSetNextReader(LeafReaderContext context) throws IOException {
-    for (BlockJoinFieldFacetAccumulator blockJoinFieldFacetAccumulator : blockJoinFieldFacetAccumulators) {
-      if(!firstSegment){
-        blockJoinFieldFacetAccumulator.migrateGlobal();
-      }
-      blockJoinFieldFacetAccumulator.setNextReader(context);
-    }
-    firstSegment = false;
-  }
-  
-  public void finish() throws IOException {
-    for (BlockJoinFieldFacetAccumulator blockJoinFieldFacetAccumulator : blockJoinFieldFacetAccumulators) {
-        blockJoinFieldFacetAccumulator.migrateGlobal();
-    }
-  }
-
-  /** is not used 
-  protected int[] includeParentDoc(int parent) {
-    final int[] docNums = ArrayUtil.grow(childTracking.getChildDocs(), childTracking.getChildCount()+1);
-    childTracking.setChildDocs(docNums); // we include parent into block, I'm not sure whether it makes sense
-    docNums[childTracking.getChildCount()]=parent;
-    return docNums;
-  }*/
-
-  protected void countFacets(final AggregatableDocIter iter) throws IOException {
-    for (BlockJoinFieldFacetAccumulator blockJoinFieldFacetAccumulator : blockJoinFieldFacetAccumulators) {
-      blockJoinFieldFacetAccumulator.updateCountsWithMatchedBlock( iter);
-    }
-  }
-  
-  NamedList getFacets() throws IOException {
-    NamedList<NamedList<Integer>> facets = new NamedList<>(blockJoinFieldFacetAccumulators.length);
-    for (BlockJoinFieldFacetAccumulator state : blockJoinFieldFacetAccumulators) {
-      facets.add(state.getFieldName(), state.getFacetValue());
-    }
-    return facets;
-  }
-  
-  
-}
diff --git a/solr/core/src/java/org/apache/solr/search/join/BlockJoinFacetComponent.java b/solr/core/src/java/org/apache/solr/search/join/BlockJoinFacetComponent.java
deleted file mode 100644
index 16f84cc..0000000
--- a/solr/core/src/java/org/apache/solr/search/join/BlockJoinFacetComponent.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.search.join;
-
-/** this is just a stub refers to {@link BlockJoinDocSetFacetComponent} to avoid
- * changes in configs */
-public class BlockJoinFacetComponent extends BlockJoinDocSetFacetComponent {
-
-}
diff --git a/solr/core/src/java/org/apache/solr/search/join/BlockJoinFacetComponentSupport.java b/solr/core/src/java/org/apache/solr/search/join/BlockJoinFacetComponentSupport.java
deleted file mode 100644
index 560e441..0000000
--- a/solr/core/src/java/org/apache/solr/search/join/BlockJoinFacetComponentSupport.java
+++ /dev/null
@@ -1,158 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.search.join;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.lucene.search.BooleanClause;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.join.ToParentBlockJoinQuery;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.params.ShardParams;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.handler.component.ResponseBuilder;
-import org.apache.solr.handler.component.SearchComponent;
-import org.apache.solr.handler.component.ShardRequest;
-import org.apache.solr.handler.component.ShardResponse;
-import org.apache.solr.request.SolrQueryRequest;
-
-abstract class BlockJoinFacetComponentSupport extends SearchComponent {
-  public static final String CHILD_FACET_FIELD_PARAMETER = "child.facet.field";
-  public static final String NO_TO_PARENT_BJQ_MESSAGE = "Block join faceting is allowed with ToParentBlockJoinQuery only";
-  public static final String COLLECTOR_CONTEXT_PARAM = "blockJoinFacetCollector";
-
-  protected void validateQuery(Query query) {
-    if (!(query instanceof ToParentBlockJoinQuery)) {
-      if (query instanceof BooleanQuery) {
-        List<BooleanClause> clauses = ((BooleanQuery) query).clauses();
-        for (BooleanClause clause : clauses) {
-          if (clause.getQuery() instanceof ToParentBlockJoinQuery) {
-            return;
-          }
-        }
-      }
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NO_TO_PARENT_BJQ_MESSAGE);
-    }
-  }
-
-  static String[] getChildFacetFields(SolrQueryRequest req) {
-    return req.getParams().getParams(CHILD_FACET_FIELD_PARAMETER);
-  }
-
-  @Override
-  public void process(ResponseBuilder rb) throws IOException {
-    if (getChildFacetFields(rb.req) != null) {
-      BlockJoinFacetAccsHolder blockJoinFacetCollector = (BlockJoinFacetAccsHolder) rb.req.getContext().get(COLLECTOR_CONTEXT_PARAM);
-      assert blockJoinFacetCollector != null;
-      NamedList output;
-      if (isShard(rb)) {
-        // distributed search, put results into own cell in order not to clash with facet component
-        output = getChildFacetFields(rb.rsp.getValues(), true);
-      } else {
-        // normal process, put results into standard response
-        output = getFacetFieldsList(rb);
-      }
-      mergeFacets(output, blockJoinFacetCollector.getFacets());
-    }
-  }
-
-  private boolean isShard(ResponseBuilder rb) {
-    return "true".equals(rb.req.getParams().get(ShardParams.IS_SHARD));
-  }
-
-  private NamedList getChildFacetFields(NamedList responseValues, boolean createIfAbsent) {
-    return getNamedListFromList(responseValues, "child_facet_fields", createIfAbsent);
-  }
-
-  private void mergeFacets(NamedList childFacetFields, NamedList shardFacets) {
-    if (shardFacets != null) {
-      for (Map.Entry<String, NamedList<Integer>> nextShardFacet : (Iterable<Map.Entry<String, NamedList<Integer>>>) shardFacets) {
-        String fieldName = nextShardFacet.getKey();
-        NamedList<Integer> collectedFacet = (NamedList<Integer>) childFacetFields.get(fieldName);
-        NamedList<Integer> shardFacet = nextShardFacet.getValue();
-        if (collectedFacet == null) {
-          childFacetFields.add(fieldName, shardFacet);
-        } else {
-          mergeFacetValues(collectedFacet, shardFacet);
-        }
-      }
-    }
-  }
-
-  private void mergeFacetValues(NamedList<Integer> collectedFacetValue, NamedList<Integer> shardFacetValue) {
-    for (Map.Entry<String, Integer> nextShardValue : shardFacetValue) {
-      String facetValue = nextShardValue.getKey();
-      Integer shardCount = nextShardValue.getValue();
-      int indexOfCollectedValue = collectedFacetValue.indexOf(facetValue, 0);
-      if (indexOfCollectedValue == -1) {
-        collectedFacetValue.add(facetValue, shardCount);
-      } else {
-        int newCount = collectedFacetValue.getVal(indexOfCollectedValue) + shardCount;
-        collectedFacetValue.setVal(indexOfCollectedValue, newCount);
-      }
-    }
-  }
-
-  private NamedList getNamedListFromList(NamedList parentList, String name, boolean createIfAbsent) {
-    NamedList result = null;
-    if (parentList != null) {
-      result = (NamedList) parentList.get(name);
-      if (result == null && createIfAbsent) {
-        result = new NamedList();
-        parentList.add(name, result);
-      }
-    }
-    return result;
-  }
-
-  @Override
-  public void handleResponses(ResponseBuilder rb, ShardRequest sreq) {
-    if ((sreq.purpose & ShardRequest.PURPOSE_GET_TOP_IDS) != 0) {
-      NamedList collectedChildFacetFields = getChildFacetFields(rb.rsp.getValues(), true);
-      List<ShardResponse> responses = sreq.responses;
-      for (ShardResponse shardResponse : responses) {
-        NamedList shardChildFacetFields = getChildFacetFields(shardResponse.getSolrResponse().getResponse(), false);
-        mergeFacets(collectedChildFacetFields, shardChildFacetFields);
-      }
-    }
-  }
-
-  @Override
-  public void finishStage(ResponseBuilder rb) {
-    if (rb.stage != ResponseBuilder.STAGE_GET_FIELDS) return;
-    NamedList childFacetFields = getChildFacetFields(rb.rsp.getValues(), true);
-    NamedList facetFields = getFacetFieldsList(rb);
-    for (Map.Entry<String, NamedList> childFacetField : (Iterable<Map.Entry<String, NamedList>>) childFacetFields) {
-     facetFields.add(childFacetField.getKey(), childFacetField.getValue());
-    }
-    rb.rsp.getValues().remove("child_facet_fields");
-  }
-
-  private NamedList getFacetFieldsList(ResponseBuilder rb) {
-    NamedList facetCounts = getNamedListFromList(rb.rsp.getValues(), "facet_counts", true);
-    return getNamedListFromList(facetCounts, "facet_fields", true);
-  }
-
-
-  @Override
-  public String getDescription() {
-    return "BlockJoin facet component";
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/search/join/BlockJoinFacetFilter.java b/solr/core/src/java/org/apache/solr/search/join/BlockJoinFacetFilter.java
deleted file mode 100644
index 5d65422..0000000
--- a/solr/core/src/java/org/apache/solr/search/join/BlockJoinFacetFilter.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.search.join;
-
-import java.util.Objects;
-
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.QueryVisitor;
-import org.apache.solr.search.DelegatingCollector;
-import org.apache.solr.search.PostFilter;
-
-class BlockJoinFacetFilter extends Query implements PostFilter {
-
-  public static final int COST = 120;
-  private DelegatingCollector blockJoinFacetCollector;
-
-  public BlockJoinFacetFilter(DelegatingCollector blockJoinFacetCollector) {
-    super();
-    this.blockJoinFacetCollector = blockJoinFacetCollector;
-  }
-
-  @Override
-  public String toString(String field) {
-    return null;
-  }
-
-  @Override
-  public DelegatingCollector getFilterCollector(IndexSearcher searcher) {
-    return blockJoinFacetCollector;
-  }
-
-  @Override
-  public boolean getCache() {
-    return false;
-  }
-
-  @Override
-  public void setCache(boolean cache) {
-
-  }
-
-  @Override
-  public int getCost() {
-    return COST;
-  }
-
-  @Override
-  public void setCost(int cost) {
-
-  }
-
-  @Override
-  public boolean getCacheSep() {
-    return false;
-  }
-
-  @Override
-  public void setCacheSep(boolean cacheSep) {
-
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    return sameClassAs(other) &&
-           equalsTo(getClass().cast(other));
-  }
-
-  private boolean equalsTo(BlockJoinFacetFilter other) {
-    return Objects.equals(blockJoinFacetCollector, other.blockJoinFacetCollector);
-  }
-
-  @Override
-  public int hashCode() {
-    return classHash() * 31 + blockJoinFacetCollector.hashCode();
-  }
-
-  @Override
-  public void visit(QueryVisitor visitor) {
-    visitor.visitLeaf(this);
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/search/join/BlockJoinFieldFacetAccumulator.java b/solr/core/src/java/org/apache/solr/search/join/BlockJoinFieldFacetAccumulator.java
deleted file mode 100644
index 703307c..0000000
--- a/solr/core/src/java/org/apache/solr/search/join/BlockJoinFieldFacetAccumulator.java
+++ /dev/null
@@ -1,235 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.search.join;
-
-import java.io.IOException;
-import java.util.Arrays;
-
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.MultiDocValues;
-import org.apache.lucene.index.OrdinalMap;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.CharsRefBuilder;
-import org.apache.lucene.util.LongValues;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.request.DocValuesFacets;
-import org.apache.solr.schema.FieldType;
-import org.apache.solr.schema.SchemaField;
-import org.apache.solr.search.DocIterator;
-import org.apache.solr.search.SolrIndexSearcher;
-
-/**
- * This class is responsible for collecting block join facet counts for particular field
- */
-class BlockJoinFieldFacetAccumulator {
-  private String fieldName;
-  private FieldType fieldType;
-  private int currentSegment = -1;
-  // for term lookups only
-  private SortedSetDocValues topSSDV;
-  private int[] globalCounts;
-  private SortedSetDocValues segmentSSDV;
-  // elems are : facet value counter<<32 | last parent doc num 
-  private long[] segmentAccums = new long[0];
-  // for mapping per-segment ords to global ones
-  private OrdinalMap ordinalMap;
-  private SchemaField schemaField;
-  private SortedDocValues segmentSDV;
-  
-  BlockJoinFieldFacetAccumulator(String fieldName, SolrIndexSearcher searcher) throws IOException {
-    this.fieldName = fieldName;
-    schemaField = searcher.getSchema().getField(fieldName);
-    fieldType = schemaField.getType();
-    ordinalMap = null;
-    if (schemaField.multiValued()) {
-      topSSDV = searcher.getSlowAtomicReader().getSortedSetDocValues(fieldName);
-      if (topSSDV instanceof MultiDocValues.MultiSortedSetDocValues) {
-        ordinalMap = ((MultiDocValues.MultiSortedSetDocValues) topSSDV).mapping;
-      }
-    } else {
-      SortedDocValues single = searcher.getSlowAtomicReader().getSortedDocValues(fieldName);
-      if (single instanceof MultiDocValues.MultiSortedDocValues) {
-        ordinalMap = ((MultiDocValues.MultiSortedDocValues) single).mapping;
-      }
-      if (single != null) {
-        topSSDV = DocValues.singleton(single);
-      }
-    }
-  }
-  
-  private boolean initSegmentData(String fieldName, LeafReaderContext leaf) throws IOException {
-    segmentSSDV = DocValues.getSortedSet(leaf.reader(), fieldName);
-    segmentAccums  = ArrayUtil.grow(segmentAccums, (int)segmentSSDV.getValueCount()+1);//+1
-    // zero counts, -1 parent
-    Arrays.fill(segmentAccums,0,(int)segmentSSDV.getValueCount()+1, 0x00000000ffffffffL);
-    segmentSDV = DocValues.unwrapSingleton(segmentSSDV);
-    return segmentSSDV.getValueCount()!=0;// perhaps we need to count "missings"?? 
-  }
-  
-  interface AggregatableDocIter extends DocIterator {
-    void reset();
-    /** a key to aggregate the current document */
-    int getAggKey();
-    
-  }
-  static class SortedIntsAggDocIterator implements AggregatableDocIter {
-    private int[] childDocs;
-    private int childCount;
-    private int parentDoc;
-    private int pos=-1;
-    
-    public SortedIntsAggDocIterator(int[] childDocs, int childCount, int parentDoc) {
-      this.childDocs = childDocs;
-      this.childCount = childCount;
-      this.parentDoc = parentDoc;
-    }
-
-    
-    @Override
-    public boolean hasNext() {
-      return pos<childCount;
-    }
-
-    @Override
-    public Integer next() {
-      return nextDoc();
-    }
-
-    @Override
-    public int nextDoc() {
-      return childDocs[pos++];
-    }
-
-    @Override
-    public float score() {
-      return 0;
-    }
-    @Override
-    public void reset() {
-      pos=0;
-    }
-    @Override
-    public int getAggKey(){
-      return parentDoc;
-    }
-  }
-
-  void updateCountsWithMatchedBlock(AggregatableDocIter iter) throws IOException {
-    if (segmentSDV != null) {
-      // some codecs may optimize SORTED_SET storage for single-valued fields
-      for (iter.reset(); iter.hasNext(); ) {
-        final int docNum = iter.nextDoc();
-        if (docNum > segmentSDV.docID()) {
-          segmentSDV.advance(docNum);
-        }
-        
-        int term;
-        if (docNum == segmentSDV.docID()) {
-          term = segmentSDV.ordValue();
-        } else {
-          term = -1;
-        }
-        accumulateTermOrd(term, iter.getAggKey());
-        //System.out.println("doc# "+docNum+" "+fieldName+" term# "+term+" tick "+Long.toHexString(segmentAccums[1+term]));
-      }
-    } else {
-      for (iter.reset(); iter.hasNext(); ) {
-        final int docNum = iter.nextDoc();
-        if (docNum > segmentSSDV.docID()) {
-          segmentSSDV.advance(docNum);
-        }
-        if (docNum == segmentSSDV.docID()) {
-          int term = (int) segmentSSDV.nextOrd();
-          do { // absent values are designated by term=-1, first iteration counts [0] as "missing", and exit, otherwise it spins 
-            accumulateTermOrd(term, iter.getAggKey());
-          } while (term>=0 && (term = (int) segmentSSDV.nextOrd()) >= 0);
-        }
-      }
-    }
-  }
-  
-  String getFieldName() {
-    return fieldName;
-  }
-  
-  /** copy paste from {@link DocValuesFacets} */
-  NamedList<Integer> getFacetValue() throws IOException {
-    NamedList<Integer> facetValue = new NamedList<>();
-    final CharsRefBuilder charsRef = new CharsRefBuilder(); // if there is no globs, take segment's ones
-    for (int i = 1; i< (globalCounts!=null ? globalCounts.length: segmentAccums.length); i++) {
-      int count = globalCounts!=null ? globalCounts[i] : (int)(segmentAccums [i]>>32);
-      if (count > 0) {
-        BytesRef term = topSSDV.lookupOrd(-1 + i);
-        fieldType.indexedToReadable(term, charsRef);
-        facetValue.add(charsRef.toString(), count);
-      }
-    }
-    return facetValue;
-  }
-  
-  // @todo we can track in max term nums to loop only changed range while migrating and labeling 
-  private void accumulateTermOrd(int term, int parentDoc) {
-    long accum = segmentAccums[1+term];
-    if(((int)(accum & 0xffffffffL))!=parentDoc)
-    {// incrementing older 32, reset smaller 32, set them to the new parent
-      segmentAccums[1+term] = ((accum +(0x1L<<32))&0xffffffffL<<32)|parentDoc;
-    }
-  }
-  
-  void setNextReader(LeafReaderContext context) throws IOException {
-    initSegmentData(fieldName, context);
-    currentSegment = context.ord;
-  }
-  
-  void migrateGlobal(){
-    if (currentSegment<0 // no hits
-        || segmentAccums.length==0 
-        || ordinalMap==null) { // single segment
-      return;
-    }
-    
-    if(globalCounts==null){
-      // it might be just a single segment 
-        globalCounts = new int[(int) ordinalMap.getValueCount()+ /*[0] for missing*/1];
-    }else{
-      assert currentSegment>=0;
-    }
-    
-    migrateGlobal(globalCounts, segmentAccums, currentSegment, ordinalMap);
-  }
-
-  /** folds counts in segment ordinal space (segCounts) into global ordinal space (counts) 
-   * copy paste-from {@link DocValuesFacets#migrateGlobal(int[], int[], int, OrdinalMap)}*/
-  void migrateGlobal(int counts[], long segCounts[], int subIndex, OrdinalMap map) {
-    
-    final LongValues ordMap = map.getGlobalOrds(subIndex);
-    // missing count
-    counts[0] += (int) (segCounts[0]>>32);
-    
-    // migrate actual ordinals
-    for (int ord = 1; ord <= segmentSSDV.getValueCount(); ord++) {
-      int count = (int) (segCounts[ord]>>32);
-      if (count != 0) {
-        counts[1+(int) ordMap.get(ord-1)] += count;
-      }
-    }
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/search/join/BlockJoinParentQParser.java b/solr/core/src/java/org/apache/solr/search/join/BlockJoinParentQParser.java
index f4bbc9f..416c9f3 100644
--- a/solr/core/src/java/org/apache/solr/search/join/BlockJoinParentQParser.java
+++ b/solr/core/src/java/org/apache/solr/search/join/BlockJoinParentQParser.java
@@ -85,6 +85,7 @@ public class BlockJoinParentQParser extends FiltersQParser {
   }
 
   public static BitDocIdSetFilterWrapper getCachedFilter(final SolrQueryRequest request, Query parentList) {
+    @SuppressWarnings("unchecked")
     SolrCache<Query, Filter> parentCache = request.getSearcher().getCache(CACHE_NAME);
     // lazily retrieve from solr cache
     BitDocIdSetFilterWrapper result;
diff --git a/solr/core/src/java/org/apache/solr/search/join/ChildFieldValueSourceParser.java b/solr/core/src/java/org/apache/solr/search/join/ChildFieldValueSourceParser.java
index b6312d5..85bd3d7 100644
--- a/solr/core/src/java/org/apache/solr/search/join/ChildFieldValueSourceParser.java
+++ b/solr/core/src/java/org/apache/solr/search/join/ChildFieldValueSourceParser.java
@@ -129,10 +129,12 @@ public class ChildFieldValueSourceParser extends ValueSourceParser {
         return new ToParentBlockJoinSortField(childField.getName(), 
             type, reverse, 
             parentFilter, childFilter) {
+          @SuppressWarnings("unchecked")
           @Override
           public FieldComparator<?> getComparator(int numHits, int sortPos) {
             final FieldComparator<?> comparator = super.getComparator(numHits, sortPos);
-            return type ==Type.STRING ?  new BytesToStringComparator((FieldComparator<BytesRef>) comparator): comparator;
+            return type ==Type.STRING ?  new BytesToStringComparator((FieldComparator<BytesRef>) comparator)
+                : comparator;
           }
         };
     }
@@ -148,7 +150,8 @@ public class ChildFieldValueSourceParser extends ValueSourceParser {
     }
 
     @Override
-    public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
+    public FunctionValues getValues(@SuppressWarnings("rawtypes") Map context,
+        LeafReaderContext readerContext) throws IOException {
       throw new UnsupportedOperationException(this + " is only for sorting");
     }
   }
diff --git a/solr/core/src/java/org/apache/solr/search/join/ScoreModeParser.java b/solr/core/src/java/org/apache/solr/search/join/ScoreModeParser.java
index 971f23f..a303046 100644
--- a/solr/core/src/java/org/apache/solr/search/join/ScoreModeParser.java
+++ b/solr/core/src/java/org/apache/solr/search/join/ScoreModeParser.java
@@ -25,6 +25,7 @@ import org.apache.lucene.search.join.ScoreMode;
 import org.apache.solr.search.SyntaxError;
 
 class ScoreModeParser {
+  @SuppressWarnings("serial")
   final private static Map<String, ScoreMode> lowerAndCapitalCase = 
       Collections.unmodifiableMap( new HashMap<String, ScoreMode>() {
     {
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-blockjoinfacetcomponent.xml b/solr/core/src/test-files/solr/collection1/conf/schema-blockjoinfacetcomponent.xml
deleted file mode 100644
index 8db75b6..0000000
--- a/solr/core/src/test-files/solr/collection1/conf/schema-blockjoinfacetcomponent.xml
+++ /dev/null
@@ -1,40 +0,0 @@
-<?xml version="1.0" ?>
-<!--
-  Licensed to the Apache Software Foundation (ASF) under one or more
-  contributor license agreements.  See the NOTICE file distributed with
-  this work for additional information regarding copyright ownership.
-  The ASF licenses this file to You under the Apache License, Version 2.0
-  (the "License"); you may not use this file except in compliance with
-  the License.  You may obtain a copy of the License at
-
-      http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
-  -->
-
-<schema name="test" version="1.0">
-  <fieldType name="int" class="${solr.tests.IntegerFieldType}" docValues="${solr.tests.numeric.dv}" precisionStep="0" positionIncrementGap="0"/>
-  <fieldType name="float" class="${solr.tests.FloatFieldType}" docValues="${solr.tests.numeric.dv}" precisionStep="0" positionIncrementGap="0"/>
-  <fieldType name="long" class="${solr.tests.LongFieldType}" docValues="${solr.tests.numeric.dv}" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
-  <fieldtype name="string" class="solr.StrField" sortMissingLast="true"/>
-
-  <field name="id" type="string" indexed="true" stored="true" multiValued="false" required="false"/>
-  <field name="_version_" type="long" indexed="true" stored="true" multiValued="false"/>
-  <field name="_root_" type="string" indexed="true" stored="true" multiValued="false" required="false"/>
-
-  <field name="name" type="string" indexed="true" stored="true"/>
-  <dynamicField name="*_s" type="string" indexed="true" stored="true" multiValued="false"/>
-
-  <!-- facet docValues fields -->
-  <dynamicField name="*_s_single" type="string" indexed="true" stored="true" docValues="true" multiValued="false"/>
-  <dynamicField name="*_s_multi" type="string" indexed="true" stored="true" docValues="true" multiValued="true"/>
-  <dynamicField name="*_i_multi" type="int" indexed="true" stored="true" docValues="true" multiValued="true"/>
-  <dynamicField name="*_f_multi" type="float" indexed="true" stored="true" docValues="true" multiValued="true"/>
-
-
-  <uniqueKey>id</uniqueKey>
-</schema>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-blockjoinfacetcomponent.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-blockjoinfacetcomponent.xml
deleted file mode 100644
index b114535..0000000
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-blockjoinfacetcomponent.xml
+++ /dev/null
@@ -1,58 +0,0 @@
-<?xml version="1.0" ?>
-
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements.  See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License.  You may obtain a copy of the License at
-
-     http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-
-<config>
-  <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-
-  <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
-  <schemaFactory class="ClassicIndexSchemaFactory"/>
-  <dataDir>${solr.data.dir:}</dataDir>
-
-  <requestHandler name="/select" class="solr.SearchHandler" />
-
-  <requestHandler name="/replication" class="solr.ReplicationHandler" startup="lazy" />
-
-  <updateHandler class="solr.DirectUpdateHandler2">
-    <updateLog>
-      <str name="dir">${solr.ulog.dir:}</str>
-    </updateLog>
-  </updateHandler>
-
-  <searchComponent name="blockJoinFacet" class="org.apache.solr.search.join.BlockJoinFacetComponent"/>
-  <searchComponent name="blockJoinDocSetFacet" class="org.apache.solr.search.join.BlockJoinDocSetFacetComponent"/>
-
-  <requestHandler name="/blockJoinFacetRH" class="org.apache.solr.handler.component.SearchHandler">
-    <lst name="defaults">
-      <str name="shards.qt">/blockJoinFacetRH</str>
-    </lst>
-    <arr name="last-components">
-      <str>blockJoinFacet</str>
-    </arr>
-  </requestHandler>
-  
-  <requestHandler name="/blockJoinDocSetFacetRH" class="org.apache.solr.handler.component.SearchHandler">
-    <lst name="defaults">
-      <str name="shards.qt">/blockJoinDocSetFacetRH</str>
-    </lst>
-    <arr name="last-components">
-      <str>blockJoinDocSetFacet</str>
-    </arr>
-  </requestHandler>
-  
-</config>
diff --git a/solr/core/src/test/org/apache/solr/search/join/BJQParserTest.java b/solr/core/src/test/org/apache/solr/search/join/BJQParserTest.java
index 48d64d6..c563563 100644
--- a/solr/core/src/test/org/apache/solr/search/join/BJQParserTest.java
+++ b/solr/core/src/test/org/apache/solr/search/join/BJQParserTest.java
@@ -336,8 +336,10 @@ public class BJQParserTest extends SolrTestCaseJ4 {
 
   
   @Test
-  public void nullInit() {
-    new BlockJoinParentQParserPlugin().init(null);
+  public void nullInit() throws Exception {
+    final BlockJoinParentQParserPlugin blockJoinParentQParserPlugin = new BlockJoinParentQParserPlugin();
+    blockJoinParentQParserPlugin.init(null);
+    blockJoinParentQParserPlugin.close();
   }
 
   private final static String eParent[] = new String[]{"//*[@numFound='1']",
diff --git a/solr/core/src/test/org/apache/solr/search/join/BlockJoinFacetDistribTest.java b/solr/core/src/test/org/apache/solr/search/join/BlockJoinFacetDistribTest.java
deleted file mode 100644
index c4f0896..0000000
--- a/solr/core/src/test/org/apache/solr/search/join/BlockJoinFacetDistribTest.java
+++ /dev/null
@@ -1,237 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.search.join;
-
-import java.io.IOException;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.response.FacetField;
-import org.apache.solr.client.solrj.response.FacetField.Count;
-import org.apache.solr.client.solrj.response.QueryResponse;
-import org.apache.solr.cloud.SolrCloudTestCase;
-import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.common.util.NamedList;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-public class BlockJoinFacetDistribTest extends SolrCloudTestCase{
-
-  private static final int defFacetLimit = 10;
-  private static final String collection = "facetcollection";
-
-  @BeforeClass
-  public static void setupCluster() throws Exception {
-    final Path configDir = Paths.get(TEST_HOME(), "collection1", "conf");
-
-    String configName = "solrCloudCollectionConfig";
-    int nodeCount = 6;
-    configureCluster(nodeCount)
-       .addConfig(configName, configDir)
-       .configure();
-    
-    
-    Map<String, String> collectionProperties = new HashMap<>();
-    collectionProperties.put("config", "solrconfig-blockjoinfacetcomponent.xml" );
-    collectionProperties.put("schema", "schema-blockjoinfacetcomponent.xml"); 
-    
-    // create a collection holding data for the "to" side of the JOIN
-    
-    int shards = 3;
-    int replicas = 2 ;
-    CollectionAdminRequest.createCollection(collection, configName, shards, replicas)
-        .setProperties(collectionProperties)
-        .process(cluster.getSolrClient());
-    
-    cluster.waitForActiveCollection(collection, shards, shards * replicas);
-
-  }
-
-  final static List<String> colors = Arrays.asList("red","blue","brown","white","black","yellow","cyan","magenta","blur",
-      "fuchsia", "light","dark","green","grey","don't","know","any","more" );
-  final static List<String> sizes = Arrays.asList("s","m","l","xl","xxl","xml","xxxl","3","4","5","6","petite","maxi");
-  
-  @SuppressWarnings("unchecked")
-  @Test 
-  public void testBJQFacetComponent() throws Exception {
-    
-    assert ! colors.removeAll(sizes): "there is no colors in sizes";
-    Collections.shuffle(colors,random());
-    List<String> matchingColors = colors.subList(0, Math.min(atLeast(random(), 2), colors.size()));
-        
-    Map<String, Set<Integer>> parentIdsByAttrValue = new HashMap<String, Set<Integer>>(){
-      @Override
-      public Set<Integer> get(Object key) {
-        return super.get(key)==null && put((String)key, new HashSet<>())==null?super.get(key):super.get(key);
-      }
-    };
-    
-    cluster.getSolrClient().deleteByQuery(collection, "*:*");
-    
-    final int parents = atLeast(10);
-    boolean aggregationOccurs = false;
-    List<SolrInputDocument> parentDocs = new ArrayList<>();
-    for(int parent=0; parent<parents || !aggregationOccurs;parent++){
-      assert parent < 2000000 : "parent num "+parent+
-           " aggregationOccurs:"+aggregationOccurs+". Sorry! too tricky loop condition.";
-      SolrInputDocument pdoc = new SolrInputDocument();
-      pdoc.addField("id", parent);
-      pdoc.addField("type_s", "parent");
-      final String parentBrand = "brand"+(random().nextInt(5));
-      pdoc.addField("BRAND_s", parentBrand);
-      
-      for(int child=0; child<atLeast(colors.size()/2);child++){
-        SolrInputDocument childDoc= new SolrInputDocument();
-        final String color = colors.get(random().nextInt(colors.size()));
-        childDoc.addField("COLOR_s", color);
-        final String size = sizes.get(random().nextInt(sizes.size()));
-        childDoc.addField("SIZE_s",  size);
-        
-        if(matchingColors.contains(color)){
-          final boolean colorDupe = !parentIdsByAttrValue.get(color).add(parent);
-          final boolean sizeDupe = !parentIdsByAttrValue.get(size).add(parent);
-          aggregationOccurs |= colorDupe || sizeDupe;
-        }
-        pdoc.addChildDocument(childDoc);
-      }
-      parentDocs.add(pdoc);
-      if (!parentDocs.isEmpty() && rarely()) {
-        indexDocs(parentDocs);
-        parentDocs.clear();
-        cluster.getSolrClient().commit(collection, false, false, true);
-      }
-    }
-    if (!parentDocs.isEmpty()) {
-      indexDocs(parentDocs);
-    }
-    if (random().nextBoolean()) {
-      cluster.getSolrClient().commit(collection);
-    } else {
-      cluster.getSolrClient().optimize(collection);
-    }
-    // to parent query
-    final String matchingColorsCommaSep = matchingColors.toString().replaceAll("[ \\[\\]]", "");
-    final String childQueryClause = "{!terms f=COLOR_s}" + matchingColorsCommaSep;
-      final boolean oldFacetsEnabled = random().nextBoolean();
-      final boolean limitJsonSizes = random().nextBoolean();
-      final boolean limitJsonColors = random().nextBoolean();
-      
-      QueryResponse results = query("q", "{!parent which=\"type_s:parent\" v=$matchingColors}",//+childQueryClause,
-          "matchingColors", childQueryClause,
-          "facet", oldFacetsEnabled ? "true":"false", // try to enforce multiple phases
-              oldFacetsEnabled ? "facet.field" : "ignore" , "BRAND_s",
-              oldFacetsEnabled&&usually() ? "facet.limit" : "ignore" , "1",
-              oldFacetsEnabled&&usually() ? "facet.mincount" : "ignore" , "2",
-              oldFacetsEnabled&&usually() ? "facet.overrequest.count" : "ignore" , "0",
-          "qt",  random().nextBoolean() ? "/blockJoinDocSetFacetRH" : "/blockJoinFacetRH",
-          "child.facet.field", "COLOR_s",
-          "child.facet.field", "SIZE_s",
-          "distrib.singlePass", random().nextBoolean() ? "true":"false",
-          "rows", random().nextBoolean() ? "0":"10",
-          "json.facet","{ "
-              + "children:{ type: query, query:\"*:*\", domain:{"
-                    +"blockChildren:\"type_s:parent\", filter:{param:matchingColors}"
-                    + "}, facet:{ colors:{ type:field, field:COLOR_s,"
-                    +              (limitJsonColors ? "":" limit:-1,")
-                    +              " facet:{ inprods:\"uniqueBlock(_root_)\"}}, "
-                    +         "sizes:{type:field, field:SIZE_s, "
-                    +              (limitJsonSizes ? "" : "limit:-1,")
-                    +              " facet:{inprods:\"uniqueBlock(_root_)\"}}"
-                    + "}"
-              + "}}", "debugQuery","true"//, "shards", "shard1"
-          );
-      NamedList<Object> resultsResponse = results.getResponse();
-      assertNotNull(resultsResponse);
-      FacetField color_s = results.getFacetField("COLOR_s");
-      FacetField size_s = results.getFacetField("SIZE_s");
-      
-      String msg = ""+parentIdsByAttrValue+" "+color_s+" "+size_s;
-      for (FacetField facet: new FacetField[]{color_s, size_s}) {
-        for (Count c : facet.getValues()) {
-          assertEquals(c.getName()+"("+msg+")", 
-              parentIdsByAttrValue.get(c.getName()).size(), c.getCount());
-        }
-      }
-
-      assertEquals(msg , parentIdsByAttrValue.size(),color_s.getValueCount() + size_s.getValueCount());
-
-      final List<NamedList<Object>> jsonSizes = (List<NamedList<Object>>)
-                              get(resultsResponse, "facets", "children", "sizes", "buckets");
-      final List<NamedList<Object>> jsonColors = (List<NamedList<Object>>)
-                                get(resultsResponse, "facets", "children", "colors", "buckets");
-
-      if (limitJsonColors) {
-        assertTrue(""+jsonColors, jsonColors.size()<=defFacetLimit);
-      }
-
-      if (limitJsonSizes) {
-        assertTrue(""+jsonSizes, jsonSizes.size()<=defFacetLimit);
-      }
-
-      for (List<NamedList<Object>> vals : new List[] { jsonSizes,jsonColors}) {
-        int i=0;
-        for(NamedList<Object> tuples: vals) {
-          String  val = (String) get(tuples,"val");
-          Number  count = (Number) get(tuples,"inprods");
-          if (((vals==jsonSizes && limitJsonSizes) || // vals close to the limit are not exact 
-              (vals==jsonColors && limitJsonColors)) && i>=defFacetLimit/2) {
-            assertTrue(i+ "th "+tuples+". "+vals, 
-                parentIdsByAttrValue.get(val).size()>= count.intValue() &&
-                count.intValue()>0);
-          } else {
-            assertEquals(tuples+". "+vals, 
-                parentIdsByAttrValue.get(val).size(),count.intValue());
-          }
-          i++;
-        }
-      }
-      if (!limitJsonColors && !limitJsonSizes) {
-        assertEquals(""+jsonSizes+jsonColors, parentIdsByAttrValue.size(),jsonSizes.size() + jsonColors.size());
-      }
-  }
-
-  private static Object get(Object nvList, String ... segments) {
-    for(String segment: segments) {
-      nvList = ((NamedList<Object>) nvList).get(segment);
-    }
-    return nvList;
-  }
-
-  private QueryResponse query(String ... arg) throws SolrServerException, IOException {
-    ModifiableSolrParams solrParams = new ModifiableSolrParams();
-    for(int i=0; i<arg.length; i+=2) {
-      solrParams.add(arg[i], arg[i+1]);
-    }
-    return cluster.getSolrClient().query(collection, solrParams);
-  }
-
-  private void indexDocs(Collection<SolrInputDocument> pdocs) throws SolrServerException, IOException {
-    cluster.getSolrClient().add(collection, pdocs);
-  }
-}
diff --git a/solr/core/src/test/org/apache/solr/search/join/BlockJoinFacetRandomTest.java b/solr/core/src/test/org/apache/solr/search/join/BlockJoinFacetRandomTest.java
deleted file mode 100644
index 1b2e37b..0000000
--- a/solr/core/src/test/org/apache/solr/search/join/BlockJoinFacetRandomTest.java
+++ /dev/null
@@ -1,637 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.search.join;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.common.SolrException;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-public class BlockJoinFacetRandomTest extends SolrTestCaseJ4 {
-  private static String handler;
-  private static final int NUMBER_OF_PARENTS = 10;
-  private static final int NUMBER_OF_VALUES = 5;
-  private static final int NUMBER_OF_CHILDREN = 5;
-  private static final String[] facetFields = {"brand", "category", "color", "size", "type"};
-  private static final String[] otherValues = {"x_", "y_", "z_"};
-  public static final String PARENT_VALUE_PREFIX = "prn_";
-  public static final String CHILD_VALUE_PREFIX = "chd_";
-
-
-  private static Facet[] facets;
-
-  @BeforeClass
-  public static void beforeClass() throws Exception {
-    initCore("solrconfig-blockjoinfacetcomponent.xml", "schema-blockjoinfacetcomponent.xml");
-    handler = random().nextBoolean() ? "/blockJoinDocSetFacetRH":"/blockJoinFacetRH";
-    facets = createFacets();
-    createIndex();
-  }
-
-  public static void createIndex() throws Exception {
-    int i = 0;
-    List<List<List<String>>> blocks = createBlocks();
-    for (List<List<String>> block : blocks) {
-      List<XmlDoc> updBlock = new ArrayList<>();
-      for (List<String> blockFields : block) {
-        blockFields.add("id");
-        blockFields.add(Integer.toString(i));
-        updBlock.add(doc(blockFields.toArray(new String[blockFields.size()])));
-        i++;
-      }
-      //got xmls for every doc. now nest all into the last one
-      XmlDoc parentDoc = updBlock.get(updBlock.size() - 1);
-      parentDoc.xml = parentDoc.xml.replace("</doc>",
-          updBlock.subList(0, updBlock.size() - 1).toString().replaceAll("[\\[\\]]", "") + "</doc>");
-      assertU(add(parentDoc));
-
-      if (random().nextBoolean()) {
-        assertU(commit());
-        // force empty segment (actually, this will no longer create an empty segment, only a new segments_n)
-        if (random().nextBoolean()) {
-          assertU(commit());
-        }
-      }
-    }
-    assertU(commit());
-    assertQ(req("q", "*:*"), "//*[@numFound='" + i + "']");
-  }
-
-  private static List<List<List<String>>> createBlocks() {
-    List<List<List<String>>> blocks = new ArrayList<>();
-    for (int i = 0; i < NUMBER_OF_PARENTS; i++) {
-      List<List<String>> block = createChildrenBlock(i, facets);
-      List<String> fieldsList = new LinkedList<>();
-      fieldsList.add("parent_s");
-      fieldsList.add(parent(i));
-      for (Facet facet : facets) {
-        for (RandomFacetValue facetValue : facet.facetValues) {
-          RandomParentPosting posting = facetValue.postings[i];
-          if (posting.parentHasOwnValue) {
-            fieldsList.add(facet.getFieldNameForIndex());
-            fieldsList.add(facetValue.facetValue);
-          } else if (facet.multiValued && random().nextBoolean()) {
-            fieldsList.add(facet.getFieldNameForIndex());
-            fieldsList.add(someOtherValue(facet.fieldType));
-          }
-        }
-        if (facet.additionalValueIsAllowedForParent(i)&&random().nextBoolean()) {
-          fieldsList.add(facet.getFieldNameForIndex());
-          fieldsList.add(someOtherValue(facet.fieldType));
-        }
-      }
-      block.add(fieldsList);
-      blocks.add(block);
-    }
-    Collections.shuffle(blocks, random());
-    return blocks;
-  }
-
-  private static List<List<String>> createChildrenBlock(int parentIndex, Facet[] facets) {
-    List<List<String>> block = new ArrayList<>();
-    for (int i = 0; i < NUMBER_OF_CHILDREN; i++) {
-      List<String> fieldsList = new LinkedList<>();
-
-      fieldsList.add("child_s");
-      fieldsList.add(child(i));
-      fieldsList.add("parentchild_s");
-      fieldsList.add(parentChild(parentIndex, i));
-      for (Facet facet : facets) {
-        for (RandomFacetValue facetValue : facet.facetValues) {
-          RandomParentPosting posting = facetValue.postings[parentIndex];
-          if (posting.childrenHaveValue[i]) {
-            fieldsList.add(facet.getFieldNameForIndex());
-            fieldsList.add(facetValue.facetValue);
-          } else if (facet.multiValued && random().nextBoolean()) {
-            fieldsList.add(facet.getFieldNameForIndex());
-            fieldsList.add(someOtherValue(facet.fieldType));
-          }
-        }
-        if (facet.additionalValueIsAllowedForChild(parentIndex,i)&&random().nextBoolean()) {
-          fieldsList.add(facet.getFieldNameForIndex());
-          fieldsList.add(someOtherValue(facet.fieldType));
-        }
-      }
-      block.add(fieldsList);
-    }
-    Collections.shuffle(block, random());
-    return block;
-  }
-
-  private static String parent(int docNumber) {
-    return fieldValue(PARENT_VALUE_PREFIX, docNumber);
-  }
-
-  private static String child(int docNumber) {
-    return fieldValue(CHILD_VALUE_PREFIX, docNumber);
-  }
-
-  private static String someOtherValue(FieldType fieldType) {
-    int randomValue = random().nextInt(NUMBER_OF_VALUES) + NUMBER_OF_VALUES;
-    switch (fieldType) {
-      case String :
-        int index = random().nextInt(otherValues.length);
-        return otherValues[index]+randomValue;
-      case Float:
-        return createFloatValue(randomValue);
-      default:
-        return String.valueOf(randomValue);
-
-    }
-
-  }
-
-  private static String createFloatValue(int intValue) {
-    return intValue + ".01";
-  }
-
-  private static String fieldValue(String valuePrefix, int docNumber) {
-    return valuePrefix + docNumber;
-  }
-
-  private static String parentChild(int parentIndex, int childIndex) {
-    return parent(parentIndex) + "_" + child(childIndex);
-  }
-
-  @AfterClass
-  public static void cleanUp() throws Exception {
-    if (null != h) {
-      assertU(delQ("*:*"));
-      optimize();
-      assertU((commit()));
-    }
-  }
-
-  @Test
-  public void testValidation() throws Exception {
-    assertQ("Component is ignored",
-        req("q", "+parent_s:(prn_1 prn_2)", "qt", handler)
-        , "//*[@numFound='2']"
-        , "//doc/str[@name=\"parent_s\"]='prn_1'"
-        , "//doc/str[@name=\"parent_s\"]='prn_2'"
-    );
-
-    assertQEx("Validation exception is expected because query is not ToParentBlockJoinQuery",
-        BlockJoinFacetComponent.NO_TO_PARENT_BJQ_MESSAGE,
-        req(
-            "q", "t",
-            "df", "name",
-            "qt", handler,
-            BlockJoinFacetComponent.CHILD_FACET_FIELD_PARAMETER, facetFields[0]
-        ),
-        SolrException.ErrorCode.BAD_REQUEST
-    );
-
-    assertQEx("Validation exception is expected because facet field is not defined in schema",
-        req(
-            "q", "{!parent which=\"parent_s:[* TO *]\"}child_s:chd_1",
-            "qt", handler,
-            BlockJoinFacetComponent.CHILD_FACET_FIELD_PARAMETER, "undefinedField"
-        ),
-        SolrException.ErrorCode.BAD_REQUEST
-    );
-  }
-
-  @Test
-  public void testAllDocs() throws Exception {
-    int[] randomFacets = getRandomArray(facets.length);
-    assertQ("Random facets for all docs should be calculated",
-        req(randomFacetsRequest(null, null, null, null, null, randomFacets)),
-        expectedResponse(null, null, randomFacets));
-  }
-
-  @Test
-  public void testRandomParentsAllChildren() throws Exception {
-    int[] randomParents = getRandomArray(NUMBER_OF_PARENTS);
-    int[] randomFacets = getRandomArray(facets.length);
-    assertQ("Random facets for random parents should be calculated",
-        req(randomFacetsRequest(randomParents, null, null, null, null, randomFacets)),
-        expectedResponse(randomParents, null, randomFacets));
-  }
-
-  @Test
-  public void testRandomChildrenAllParents() throws Exception {
-    int[] randomChildren = getRandomArray(NUMBER_OF_CHILDREN);
-    int[] randomFacets = getRandomArray(facets.length);
-    assertQ("Random facets for all parent docs should be calculated",
-        req(randomFacetsRequest(null, randomChildren, null, null, null, randomFacets)),
-        expectedResponse(null, randomChildren, randomFacets));
-  }
-
-  @Test
-  public void testRandomChildrenRandomParents() throws Exception {
-    int[] randomParents = getRandomArray(NUMBER_OF_PARENTS);
-    int[] randomChildren = getRandomArray(NUMBER_OF_CHILDREN);
-    int[] randomFacets = getRandomArray(facets.length);
-    assertQ("Random facets for all parent docs should be calculated",
-        req(randomFacetsRequest(randomParents, randomChildren, null, null, null, randomFacets)),
-        expectedResponse(randomParents, randomChildren, randomFacets));
-  }
-
-  @Test
-  public void testRandomChildrenRandomParentsRandomRelations() throws Exception {
-    int[] randomParents = getRandomArray(NUMBER_OF_PARENTS);
-    int[] randomChildren = getRandomArray(NUMBER_OF_CHILDREN);
-    int[] parentRelations = getRandomArray(NUMBER_OF_PARENTS);
-    int[] childRelations = getRandomArray(NUMBER_OF_CHILDREN);
-    int[] randomFacets = getRandomArray(facets.length);
-    assertQ("Random facets for all parent docs should be calculated",
-        req(randomFacetsRequest(randomParents, randomChildren, parentRelations, childRelations, null, randomFacets)),
-        expectedResponse(intersection(randomParents, parentRelations),
-            intersection(randomChildren, childRelations), randomFacets));
-  }
-
-  @Test
-  public void testRandomFilters() throws Exception {
-    int[] randomParents = getRandomArray(NUMBER_OF_PARENTS);
-    int[] randomChildren = getRandomArray(NUMBER_OF_CHILDREN);
-    int[] parentRelations = getRandomArray(NUMBER_OF_PARENTS);
-    int[] childRelations = getRandomArray(NUMBER_OF_CHILDREN);
-    int[] randomParentFilters = getRandomArray(NUMBER_OF_PARENTS);
-    int[] randomFacets = getRandomArray(facets.length);
-    assertQ("Random facets for all parent docs should be calculated",
-        req(randomFacetsRequest(randomParents, randomChildren, parentRelations, childRelations, randomParentFilters, randomFacets)),
-        expectedResponse(intersection(intersection(randomParents, parentRelations), randomParentFilters),
-            intersection(randomChildren, childRelations), randomFacets));
-  }
-
-  private int[] intersection(int[] firstArray, int[] secondArray) {
-    Set<Integer> firstSet = new HashSet<>();
-    for (int i : firstArray) {
-      firstSet.add(i);
-    }
-    Set<Integer> secondSet = new HashSet<>();
-    for (int i : secondArray) {
-      secondSet.add(i);
-    }
-    firstSet.retainAll(secondSet);
-    int[] result = new int[firstSet.size()];
-    int i = 0;
-    for (Integer integer : firstSet) {
-      result[i++] = integer;
-    }
-    return result;
-  }
-
-  private String[] randomFacetsRequest(int[] parents, int[] children,
-                                       int[] parentRelations, int[] childRelations,
-                                       int[] parentFilters, int[] facetNumbers) {
-    List<String> params = new ArrayList<>(Arrays.asList(
-        "q", parentsQuery(parents),
-        "qt",handler,
-        "pq","parent_s:[* TO *]",
-        "chq", childrenQuery(children, parentRelations, childRelations),
-        "fq", flatQuery(parentFilters, "parent_s", PARENT_VALUE_PREFIX)
-        ));
-      for (int facetNumber : facetNumbers) {
-        params .add(BlockJoinFacetComponent.CHILD_FACET_FIELD_PARAMETER);
-        params .add(facets[facetNumber].getFieldNameForIndex());
-      }
-    return params.toArray(new String[params.size()]);
-  }
-
-  private String parentsQuery(int[] parents) {
-    String result;
-    if (parents == null) {
-      result = "{!parent which=$pq v=$chq}";
-    } else {
-      result = flatQuery(parents, "parent_s", PARENT_VALUE_PREFIX) + " +_query_:\"{!parent which=$pq v=$chq}\"";
-    }
-    return result;
-  }
-
-  private String flatQuery(int[] docNumbers, final String fieldName, String fieldValuePrefix) {
-    String result;
-    if (docNumbers == null) {
-      result = "+" + fieldName + ":[* TO *]";
-    } else {
-      StringBuilder builder = new StringBuilder("+" + fieldName +":(");
-      if (docNumbers.length == 0) {
-        builder.append("match_nothing_value");
-      } else {
-        for (int docNumber : docNumbers) {
-          builder.append(fieldValue(fieldValuePrefix, docNumber));
-          builder.append(" ");
-        }
-        builder.deleteCharAt(builder.length() - 1);
-      }
-      builder.append(")");
-      result = builder.toString();
-    }
-    return result;
-  }
-
-  private String childrenQuery(int[] children, int[] parentRelations, int[] childRelations) {
-    StringBuilder builder = new StringBuilder();
-    builder.append(flatQuery(children, "child_s", CHILD_VALUE_PREFIX));
-    if (parentRelations == null) {
-      if (childRelations == null) {
-        builder.append(" +parentchild_s:[* TO *]");
-      } else {
-        builder.append(" +parentchild_s:(");
-        if (childRelations.length == 0) {
-          builder.append("match_nothing_value");
-        } else {
-          for (int childRelation : childRelations) {
-            for (int i = 0; i < NUMBER_OF_PARENTS; i++) {
-              builder.append(parentChild(i, childRelation));
-              builder.append(" ");
-            }
-          }
-          builder.deleteCharAt(builder.length() - 1);
-        }
-        builder.append(")");
-      }
-    } else {
-      builder.append(" +parentchild_s:(");
-      if (parentRelations.length == 0) {
-        builder.append("match_nothing_value");
-      } else {
-        if (childRelations == null) {
-          for (int parentRelation : parentRelations) {
-              for (int i = 0; i < NUMBER_OF_CHILDREN; i++) {
-                builder.append(parentChild(parentRelation, i));
-                builder.append(" ");
-              }
-          }
-        } else if (childRelations.length == 0) {
-          builder.append("match_nothing_value");
-        } else {
-          for (int parentRelation : parentRelations) {
-
-              for (int childRelation : childRelations) {
-                builder.append(parentChild(parentRelation, childRelation));
-                builder.append(" ");
-              }
-          }
-          builder.deleteCharAt(builder.length() - 1);
-        }
-      }
-      builder.append(")");
-    }
-    return builder.toString();
-  }
-
-  private String[] expectedResponse(int[] parents, int[] children, int[] facetNumbers) {
-    List<String> result = new LinkedList<>();
-    if (children != null && children.length == 0) {
-      result.add("//*[@numFound='" + 0 + "']");
-    } else {
-      if (parents == null) {
-        result.add("//*[@numFound='" + NUMBER_OF_PARENTS + "']");
-        for (int i = 0; i < NUMBER_OF_PARENTS; i++) {
-          result.add("//doc/str[@name=\"parent_s\"]='" + parent(i) + "'");
-        }
-      } else {
-        result.add("//*[@numFound='" + parents.length + "']");
-        for (int parent : parents) {
-          result.add("//doc/str[@name=\"parent_s\"]='" + parent(parent) + "'");
-        }
-      }
-    }
-    if (facetNumbers != null) {
-      for (int facetNumber : facetNumbers) {
-        result.add("//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + facets[facetNumber].getFieldNameForIndex() + "']");
-        RandomFacetValue[] facetValues = facets[facetNumber].facetValues;
-        for (RandomFacetValue facetValue : facetValues) {
-          int expectedFacetCount = facetValue.getFacetCount(parents, children);
-          if (expectedFacetCount > 0) {
-            result.add("//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" +
-            facets[facetNumber].getFieldNameForIndex() + "']/int[@name='" + 
-                facetValue.facetValue + "' and text()='" + expectedFacetCount + "']");
-          }
-        }
-      }
-    }
-    return result.toArray(new String[result.size()]);
-  }
-
-  private static Facet[] createFacets() {
-    int[] facetsToCreate = getRandomArray(facetFields.length);
-    Facet[] facets = new Facet[facetsToCreate.length];
-    int i = 0;
-    for (int facetNumber : facetsToCreate) {
-      facets[i++] = new Facet(facetFields[facetNumber]);
-    }
-    return facets;
-  }
-
-  private static int[] getRandomArray(int maxNumber) {
-    int[] buffer = new int[maxNumber];
-    int count = 0;
-    for (int i = 0; i < maxNumber; i++) {
-      if (random().nextBoolean()) {
-        buffer[count++] = i;
-      }
-    }
-    int[] result = new int[count];
-    System.arraycopy(buffer, 0, result, 0, count);
-    return result;
-  }
-
-  private static class Facet {
-    private String fieldName;
-    private boolean multiValued = true;
-    FieldType fieldType;
-    RandomFacetValue[] facetValues;
-
-    Facet(String fieldName) {
-      this.fieldName = fieldName;
-      fieldType = FieldType.values()[random().nextInt(FieldType.values().length)];
-      if ( FieldType.String.equals(fieldType)) {
-        // sortedDocValues are supported for string fields only
-        multiValued = random().nextBoolean();
-      }
-
-      fieldType = FieldType.String;
-      facetValues = new RandomFacetValue[NUMBER_OF_VALUES];
-      for (int i = 0; i < NUMBER_OF_VALUES; i++) {
-        String value = createRandomValue(i);
-        facetValues[i] = new RandomFacetValue(value);
-      }
-      if (!multiValued) {
-        makeValuesSingle();
-      }
-    }
-
-    private String createRandomValue(int i) {
-      switch( fieldType ) {
-        case String:
-          return fieldName.substring(0, 2) + "_" + i;
-        case Float:
-          return createFloatValue(i);
-        default:
-          return String.valueOf(i);
-      }
-    }
-
-    String getFieldNameForIndex() {
-      String multiValuedPostfix = multiValued ? "_multi" : "_single";
-      return fieldName + fieldType.fieldPostfix + multiValuedPostfix;
-    }
-
-    private void makeValuesSingle() {
-      for ( int i = 0; i < NUMBER_OF_PARENTS; i++) {
-        List<Integer> values = getValuesForParent(i);
-        if ( values.size() > 0) {
-          int singleValueOrd = values.get(random().nextInt(values.size()));
-          setSingleValueForParent(i,singleValueOrd);
-        }
-        for ( int j=0; j < NUMBER_OF_CHILDREN; j++) {
-          values = getValuesForChild(i,j);
-          if ( values.size() > 0 ) {
-            int singleValueOrd = values.get(random().nextInt(values.size()));
-            setSingleValueForChild(i, j, singleValueOrd);
-          }
-        }
-      }
-    }
-
-    private List<Integer> getValuesForParent(int parentNumber) {
-      List<Integer> result = new ArrayList<>();
-      for (int i = 0; i<NUMBER_OF_VALUES; i++) {
-        if (facetValues[i].postings[parentNumber].parentHasOwnValue) {
-          result.add(i);
-        }
-      }
-      return result;
-    }
-
-    private void setSingleValueForParent(int parentNumber, int valueOrd) {
-      for (int i = 0; i<NUMBER_OF_VALUES; i++) {
-        facetValues[i].postings[parentNumber].parentHasOwnValue = (i == valueOrd);
-      }
-    }
-
-    boolean additionalValueIsAllowedForParent(int parentNumber) {
-      return multiValued || getValuesForParent(parentNumber).size() == 0;
-    }
-
-    private List<Integer> getValuesForChild(int parentNumber, int childNumber) {
-      List<Integer> result = new ArrayList<>();
-      for (int i = 0; i<NUMBER_OF_VALUES; i++) {
-        if (facetValues[i].postings[parentNumber].childrenHaveValue[childNumber]) {
-          result.add(i);
-        }
-      }
-      return result;
-    }
-
-    private void setSingleValueForChild(int parentNumber, int childNumber, int valueOrd) {
-      for (int i = 0; i<NUMBER_OF_VALUES; i++) {
-        facetValues[i].postings[parentNumber].childrenHaveValue[childNumber] = (i == valueOrd);
-      }
-    }
-
-    boolean additionalValueIsAllowedForChild(int parentNumber, int childNumber) {
-      return multiValued || getValuesForChild(parentNumber,childNumber).size() == 0;
-    }
-  }
-
-  private static class RandomFacetValue {
-    final String facetValue;
-    // rootDoc, level, docsOnLevel
-    RandomParentPosting[] postings;
-
-
-    public RandomFacetValue(String facetValue) {
-      this.facetValue = facetValue;
-      postings = new RandomParentPosting[NUMBER_OF_PARENTS];
-      for (int i = 0; i < NUMBER_OF_PARENTS; i++) {
-        postings[i] = new RandomParentPosting(random().nextBoolean());
-      }
-    }
-
-    int getFacetCount(int[] parentNumbers, int[] childNumbers) {
-      int result = 0;
-      if (parentNumbers != null) {
-        for (int parentNumber : parentNumbers) {
-          if (postings[parentNumber].isMatched(childNumbers)) {
-            result++;
-          }
-        }
-      } else {
-        for (int i = 0; i < NUMBER_OF_PARENTS; i++) {
-          if (postings[i].isMatched(childNumbers)) {
-            result++;
-          }
-        }
-      }
-      return result;
-    }
-  }
-
-  private enum  FieldType {
-    Integer("_i"),
-    Float("_f"),
-    String("_s");
-    private final String fieldPostfix;
-
-    FieldType(String fieldPostfix) {
-      this.fieldPostfix = fieldPostfix;
-    }
-  }
-
-  private static class RandomParentPosting {
-    boolean parentHasOwnValue;
-    boolean[] childrenHaveValue;
-
-    RandomParentPosting(boolean expected) {
-      childrenHaveValue = new boolean[NUMBER_OF_CHILDREN];
-      if (expected) {
-        // don't count parents
-        parentHasOwnValue = false;// random().nextBoolean();
-        if (random().nextBoolean()) {
-          for (int i = 0; i < NUMBER_OF_CHILDREN; i++) {
-            childrenHaveValue[i] = random().nextBoolean();
-          }
-        }
-      }
-    }
-
-    boolean isMatched(int[] childNumbers) {
-      boolean result = parentHasOwnValue && (childNumbers == null || childNumbers.length > 0);
-      if (!result) {
-        if (childNumbers == null) {
-          for (boolean childHasValue : childrenHaveValue) {
-            result = childHasValue;
-            if (result) {
-              break;
-            }
-          }
-        } else {
-          for (int child : childNumbers) {
-            result = childrenHaveValue[child];
-            if (result) {
-              break;
-            }
-          }
-        }
-      }
-      return result;
-    }
-  }
-}
diff --git a/solr/core/src/test/org/apache/solr/search/join/BlockJoinFacetSimpleTest.java b/solr/core/src/test/org/apache/solr/search/join/BlockJoinFacetSimpleTest.java
deleted file mode 100644
index c4d3a64..0000000
--- a/solr/core/src/test/org/apache/solr/search/join/BlockJoinFacetSimpleTest.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.search.join;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.request.SolrQueryRequest;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-public class BlockJoinFacetSimpleTest extends SolrTestCaseJ4 {
-  private static String handler;
-
-  @BeforeClass
-  public static void beforeClass() throws Exception {
-    initCore("solrconfig-blockjoinfacetcomponent.xml", "schema-blockjoinfacetcomponent.xml");
-    handler = random().nextBoolean() ? "/blockJoinDocSetFacetRH":"/blockJoinFacetRH";
-    createIndex();
-  }
-
-  public static void createIndex() throws Exception {
-    
-    final String match;
-    List<String> docs = Arrays.asList(
-// match
-    match = adoc("id", "10","type_s", "parent","BRAND_s", "Nike").replace("</doc>", 
-    ""+
-    doc("id", "11","type_s", "child","COLOR_s", "Red","SIZE_s", "XL")+// matches child filter 
-    doc("id", "12","type_s", "child","COLOR_s", "Red","SIZE_s", "XL")+// matches child filter
-    doc("id", "13","type_s", "child","COLOR_s", "Blue","SIZE_s", "XL")+"</doc>"),
-// mismatch
-    adoc("id", "100","type_s", "parent","BRAND_s", "Reebok").replace("</doc>", 
-    ""+doc("id", "101","type_s", "child","COLOR_s", "Red","SIZE_s", "M")+
-    doc("id", "102","type_s", "child","COLOR_s", "Blue","SIZE_s", "XL")+
-    doc("id", "104","type_s", "child","COLOR_s", "While","SIZE_s", "XL")+
-    doc("id", "105","type_s", "child","COLOR_s", "Green","SIZE_s", "XXXL")+
-    "</doc>"));
-    
-    Collections.shuffle(docs, random());
-    for(String d : docs){
-      assertU(d);
-    }
-    if(random().nextBoolean()){// let's have a deleted doc
-      if(random().nextBoolean()){
-        assertU("let's have two segs",commit());
-      }
-      assertU("overriding matching doc",match);
-    }
-    assertU(commit());
-    assertQ(req("q", "*:*"), "//*[@numFound='" + 9 + "']");
-  }
-
-  @Test
-  public void testSimple() throws Exception {
-    //query
-    // parents
-    assertQ(req("q", "type_s:parent"), "//*[@numFound='" + 2 + "']");
-    
-    String alt[][] ={ {"q", "{!parent which=\"type_s:parent\"}+COLOR_s:Red +SIZE_s:XL"},
-        {"q", "+{!parent which=\"type_s:parent\"}+COLOR_s:Red +BRAND_s:Nike"},
-        {"q", "{!parent which=\"type_s:parent\"}+COLOR_s:Red", "fq", "BRAND_s:Nike"}};
-    
-    for(String param[] : alt){
-      final List<String> reqParams = new ArrayList<>(Arrays.asList(param));
-      reqParams.addAll(Arrays.asList("qt",handler,
-          "facet", (random().nextBoolean() ? "true":"false"),// it's indifferent to 
-              "child.facet.field", "COLOR_s",
-              "child.facet.field", "SIZE_s"));
-      assertQ(req(reqParams.toArray(new String[0])),
-          "//*[@numFound='" + 1 + "']",
-          "//lst[@name='COLOR_s']/int[@name='Red'][.='1']",
-        //  "//lst[@name='COLOR_s']/int[@name='Blue'][.='1']",
-          "count(//lst[@name='COLOR_s']/int)=1",
-          "//lst[@name='SIZE_s']/int[@name='XL'][.='1']",
-          "count(//lst[@name='SIZE_s']/int)=1");
-      
-    }
-  }
-
-  @Test
-  public void testParentLevelFQExclusion() {
-    SolrQueryRequest req = req(
-        "qt", handler,
-        "q", "{!parent which=type_s:parent}+SIZE_s:XL",
-        "fq", "{!term f=BRAND_s tag=rbrand}Nike",
-        "facet", "true",
-        "facet.field", "BRAND_s",
-        "child.facet.field", "COLOR_s");
-    assertQ("no exclusion, brand facet got only one Nike",req, "//*[@numFound='" + 1 + "']",
-        "count(//lst[@name='BRAND_s']/int[.='1'])=1");
-  
-    assertQ("nike filter is excluded, expecting both brand in facet",req(
-        "qt", handler,
-        "q", "{!parent which=type_s:parent}+SIZE_s:XL",
-        "fq", "{!term f=BRAND_s tag=rbrand}Nike",
-        "facet", "true",
-        "facet.field", "{!ex=rbrand}BRAND_s",
-        "child.facet.field", "COLOR_s"),
-     "//*[@numFound='" + 1 + "']",
-        "count(//lst[@name='BRAND_s']/int[.='1'])=2");
-  
-  }
-}
diff --git a/solr/core/src/test/org/apache/solr/search/join/TestScoreJoinQPNoScore.java b/solr/core/src/test/org/apache/solr/search/join/TestScoreJoinQPNoScore.java
index a780d14..afa45b9 100644
--- a/solr/core/src/test/org/apache/solr/search/join/TestScoreJoinQPNoScore.java
+++ b/solr/core/src/test/org/apache/solr/search/join/TestScoreJoinQPNoScore.java
@@ -206,6 +206,7 @@ public class TestScoreJoinQPNoScore extends SolrTestCaseJ4 {
       return " score="+vals[random().nextInt(vals.length)]+" ";
   }
 
+  @SuppressWarnings({"rawtypes", "unchecked"})
   @Test
   public void testRandomJoin() throws Exception {
     int indexIter=50 * RANDOM_MULTIPLIER;
@@ -322,7 +323,7 @@ public class TestScoreJoinQPNoScore extends SolrTestCaseJ4 {
           final Map<String,String> ps = ((MapSolrParams)req.getParams()).getMap();
           final String q = ps.get("q");
           ps.put("q", q.replaceAll("\\}", " cache=false\\}"));
-          String rsp = h.query(req);
+          h.query(req);
           }
           fail(err);
         }
@@ -331,6 +332,7 @@ public class TestScoreJoinQPNoScore extends SolrTestCaseJ4 {
     }
   }
 
+  @SuppressWarnings("rawtypes")
   Map<Comparable, Set<Comparable>> createJoinMap(Map<Comparable, Doc> model, String fromField, String toField) {
     Map<Comparable, Set<Comparable>> id_to_id = new HashMap<Comparable, Set<Comparable>>();
 
@@ -357,6 +359,7 @@ public class TestScoreJoinQPNoScore extends SolrTestCaseJ4 {
   }
 
 
+  @SuppressWarnings("rawtypes")
   Set<Comparable> join(Collection<Doc> input, Map<Comparable, Set<Comparable>> joinMap) {
     Set<Comparable> ids = new HashSet<Comparable>();
     for (Doc doc : input) {
diff --git a/solr/core/src/test/org/apache/solr/search/join/TestScoreJoinQPScore.java b/solr/core/src/test/org/apache/solr/search/join/TestScoreJoinQPScore.java
index 8ce6a27..0dc0ccf 100644
--- a/solr/core/src/test/org/apache/solr/search/join/TestScoreJoinQPScore.java
+++ b/solr/core/src/test/org/apache/solr/search/join/TestScoreJoinQPScore.java
@@ -203,6 +203,7 @@ public class TestScoreJoinQPScore extends SolrTestCaseJ4 {
 
     Map<String, Metric> metrics = h.getCoreContainer().getMetricManager().registry(h.getCore().getCoreMetricManager().getRegistryName()).getMetrics();
 
+    @SuppressWarnings("rawtypes")
     MetricsMap mm = (MetricsMap)((SolrMetricManager.GaugeWrapper)metrics.get("CACHE.searcher.queryResultCache")).getGauge();
     {
       Map<String,Object> statPre = mm.getValue();
@@ -258,13 +259,14 @@ public class TestScoreJoinQPScore extends SolrTestCaseJ4 {
     // this queries are not overlap, with other in this test case. 
     // however it might be better to extract this method into the separate suite
     // for a while let's nuke a cache content, in case of repetitions
+    @SuppressWarnings("rawtypes")
     SolrCache cache = (SolrCache)h.getCore().getInfoRegistry().get("queryResultCache");
     cache.clear();
   }
 
   private ScoreMode not(ScoreMode s) {
     Random r = random();
-    final List<ScoreMode> l = new ArrayList(Arrays.asList(ScoreMode.values()));
+    final List<ScoreMode> l = new ArrayList<>(Arrays.asList(ScoreMode.values()));
     l.remove(s);
     return l.get(r.nextInt(l.size()));
   }
diff --git a/solr/core/src/test/org/apache/solr/search/join/another/BJQFilterAccessibleTest.java b/solr/core/src/test/org/apache/solr/search/join/another/BJQFilterAccessibleTest.java
index 6764f75..96ac205 100644
--- a/solr/core/src/test/org/apache/solr/search/join/another/BJQFilterAccessibleTest.java
+++ b/solr/core/src/test/org/apache/solr/search/join/another/BJQFilterAccessibleTest.java
@@ -45,7 +45,8 @@ public class BJQFilterAccessibleTest  extends SolrTestCaseJ4 {
     try (SolrQueryRequest req = lrf.makeRequest()) {
       TermQuery childQuery = new TermQuery(new Term("child_s", "l"));
       Query parentQuery = new WildcardQuery(new Term("parent_s", "*"));
-      ToParentBlockJoinQuery tpbjq = new ToParentBlockJoinQuery(childQuery, BlockJoinParentQParser.getCachedFilter(req,parentQuery).getFilter(), ScoreMode.Max);
+      ToParentBlockJoinQuery tpbjq = new ToParentBlockJoinQuery(childQuery,
+          BlockJoinParentQParser.getCachedFilter(req,parentQuery).getFilter(), ScoreMode.Max);
       Assert.assertEquals(6, req.getSearcher().search(tpbjq,10).totalHits.value);
     }
   }
diff --git a/solr/solr-ref-guide/src/blockjoin-faceting.adoc b/solr/solr-ref-guide/src/blockjoin-faceting.adoc
deleted file mode 100644
index 94b5110..0000000
--- a/solr/solr-ref-guide/src/blockjoin-faceting.adoc
+++ /dev/null
@@ -1,115 +0,0 @@
-= BlockJoin Faceting
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-
-BlockJoin facets allow you to aggregate children facet counts by their parents.
-
-It is a common requirement that if a parent document has several children documents, all of them need to increment facet value count only once. This functionality is provided by `BlockJoinDocSetFacetComponent`, and `BlockJoinFacetComponent` just an alias for compatibility.
-
-CAUTION: This functionality is considered deprecated. Users are encouraged to use `uniqueBlock(\_root_)` aggregation under a `terms` facet in the <<json-faceting-domain-changes.adoc#block-join-domain-changes,JSON Facet API>>.
-If this component is used, it must be explicitly enabled for a request handler in `solrconfig.xml`, in the same way as any other <<requesthandlers-and-searchcomponents-in-solrconfig.adoc#requesthandlers-and-searchcomponents-in-solrconfig,search component>>.
-
-This example shows how you could add this search components to `solrconfig.xml` and define it in request handler:
-
-[source,xml]
-----
-  <searchComponent name="bjqFacetComponent" class="org.apache.solr.search.join.BlockJoinDocSetFacetComponent"/>
-
-   <requestHandler name="/bjqfacet" class="org.apache.solr.handler.component.SearchHandler">
-    <lst name="defaults">
-      <str name="shards.qt">/bjqfacet</str>
-    </lst>
-    <arr name="last-components">
-      <str>bjqFacetComponent</str>
-    </arr>
-  </requestHandler>
-----
-
-This component can be added into any search request handler. This component work with distributed search in SolrCloud mode.
-
-Documents should be added in children-parent blocks as described in <<indexing-nested-documents.adoc#indexing-nested-documents,indexing nested child documents>>. Examples:
-
-.Sample document
-[source,xml]
-----
-<add>
-  <doc>
-    <field name="id">1</field>
-    <field name="type_s">parent</field>
-    <doc>
-      <field name="id">11</field>
-      <field name="COLOR_s">Red</field>
-      <field name="SIZE_s">XL</field>
-      <field name="PRICE_i">6</field>
-    </doc>
-    <doc>
-      <field name="id">12</field>
-      <field name="COLOR_s">Red</field>
-      <field name="SIZE_s">XL</field>
-      <field name="PRICE_i">7</field>
-    </doc>
-    <doc>
-      <field name="id">13</field>
-      <field name="COLOR_s">Blue</field>
-      <field name="SIZE_s">L</field>
-      <field name="PRICE_i">5</field>
-    </doc>
-  </doc>
-  <doc>
-    <field name="id">2</field>
-    <field name="type_s">parent</field>
-    <doc>
-      <field name="id">21</field>
-      <field name="COLOR_s">Blue</field>
-      <field name="SIZE_s">XL</field>
-      <field name="PRICE_i">6</field>
-    </doc>
-    <doc>
-      <field name="id">22</field>
-      <field name="COLOR_s">Blue</field>
-      <field name="SIZE_s">XL</field>
-      <field name="PRICE_i">7</field>
-    </doc>
-    <doc>
-      <field name="id">23</field>
-      <field name="COLOR_s">Red</field>
-      <field name="SIZE_s">L</field>
-      <field name="PRICE_i">5</field>
-    </doc>
-  </doc>
-</add>
-----
-
-Queries are constructed the same way as for a <<other-parsers.adoc#block-join-query-parsers,Parent Block Join query>>. For example:
-
-[source,text]
-----
-http://localhost:8983/solr/bjqfacet?q={!parent which=type_s:parent}SIZE_s:XL&child.facet.field=COLOR_s
-----
-
-As a result we should have facets for Red(1) and Blue(1), because matches on children `id=11` and `id=12` are aggregated into single hit into parent with `id=1`.
-
-The key components of the request shown above are:
-
-`/bjqfacet?`::
-The name of the request handler that has been defined with a block join facet component enabled.
-
-`q={!parent which=type_s:parent}SIZE_s:XL`::
-The mandatory parent query as a main query. The parent query could also be a subordinate clause in a more complex query.
-
-`&child.facet.field=COLOR_s`::
-The child document field, which might be repeated many times with several fields, as necessary.
diff --git a/solr/solr-ref-guide/src/faceting.adoc b/solr/solr-ref-guide/src/faceting.adoc
index efbb033..5473117 100644
--- a/solr/solr-ref-guide/src/faceting.adoc
+++ b/solr/solr-ref-guide/src/faceting.adoc
@@ -1,5 +1,4 @@
 = Faceting
-:page-children: blockjoin-faceting
 // Licensed to the Apache Software Foundation (ASF) under one
 // or more contributor license agreements.  See the NOTICE file
 // distributed with this work for additional information
diff --git a/solr/solr-ref-guide/src/json-faceting-domain-changes.adoc b/solr/solr-ref-guide/src/json-faceting-domain-changes.adoc
index 437c9a2..473f82f 100644
--- a/solr/solr-ref-guide/src/json-faceting-domain-changes.adoc
+++ b/solr/solr-ref-guide/src/json-faceting-domain-changes.adoc
@@ -179,7 +179,7 @@ NOTE: While a `query` domain can be combined with an additional domain `filter`,
 
 When a collection contains <<indexing-nested-documents.adoc#indexing-nested-documents, Nested Documents>>, the `blockChildren` or `blockParent` domain options can be used transform an existing domain containing one type of document, into a domain containing the documents with the specified relationship (child or parent of) to the documents from the original domain.
 
-Both of these options work similarly to the corresponding <<other-parsers.adoc#block-join-query-parsers,Block Join Query Parsers>> by taking in a single String query that exclusively matches all parent documents in the collection.  If `blockParent` is used, then the resulting domain will contain all parent documents of the children from the original domain.  If `blockChildren` is used, then the resulting domain will contain all child documents of the parents from the original domain.
+Both of these options work similarly to the corresponding <<other-parsers.adoc#block-join-query-parsers,Block Join Query Parsers>> by taking in a single String query that exclusively matches all parent documents in the collection.  If `blockParent` is used, then the resulting domain will contain all parent documents of the children from the original domain.  If `blockChildren` is used, then the resulting domain will contain all child documents of the parents from the original domain. Qui [...]
 
 [source,json,subs="verbatim,callouts"]]
 ----
diff --git a/solr/solr-ref-guide/src/searching-nested-documents.adoc b/solr/solr-ref-guide/src/searching-nested-documents.adoc
index 20edbf5..18e6420 100644
--- a/solr/solr-ref-guide/src/searching-nested-documents.adoc
+++ b/solr/solr-ref-guide/src/searching-nested-documents.adoc
@@ -25,7 +25,7 @@ section for more details about schema and index configuration.
 
 [NOTE]
 This section does not show case faceting on nested documents. For nested document faceting, please refer to the
-<<blockjoin-faceting#blockjoin-faceting, Block Join Faceting>> section.
+<<json-facet-api#uniqueblock-and-block-join-counts, Block Join Facet Counts>> section.
 
 == Query Examples