You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2017/05/08 15:26:24 UTC

[07/50] [abbrv] lucene-solr:jira/solr-10262: SOLR-10583: JSON Faceting now supports a query time 'join' domain change option

SOLR-10583: JSON Faceting now supports a query time 'join' domain change option


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/15e1c5d3
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/15e1c5d3
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/15e1c5d3

Branch: refs/heads/jira/solr-10262
Commit: 15e1c5d34f69fa2662b5299dce6fc808854f8ba3
Parents: 17563ce
Author: Chris Hostetter <ho...@apache.org>
Authored: Wed May 3 10:30:02 2017 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Wed May 3 10:30:02 2017 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../apache/solr/search/JoinQParserPlugin.java   |  22 +-
 .../solr/search/facet/FacetProcessor.java       |  10 +
 .../apache/solr/search/facet/FacetRequest.java  |  74 ++-
 .../cloud/TestCloudJSONFacetJoinDomain.java     | 596 +++++++++++++++++++
 .../search/facet/TestJsonFacetRefinement.java   |  50 +-
 .../solr/search/facet/TestJsonFacets.java       | 239 ++++++++
 7 files changed, 987 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/15e1c5d3/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 75f54c1..7eaafae 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -204,6 +204,8 @@ New Features
 
 * SOLR-10430: Add ls command to ZkCLI for listing only sub-directories. (Peter Szantai-Kis via Mark Miller)
 
+* SOLR-10583: JSON Faceting now supports a query time 'join' domain change option (hossman)
+
 Optimizations
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/15e1c5d3/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
index fca9a34..abdea53 100644
--- a/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
@@ -121,21 +121,37 @@ public class JoinQParserPlugin extends QParserPlugin {
     };
   }
 
+  /**
+   * A helper method for other plugins to create (non-scoring) JoinQueries wrapped around arbitrary queries against the same core.
+   * 
+   * @param subQuery the query to define the starting set of documents on the "left side" of the join
+   * @param fromField "left side" field name to use in the join
+   * @param toField "right side" field name to use in the join
+   */
+  public static Query createJoinQuery(Query subQuery, String fromField, String toField) {
+    return new JoinQuery(fromField, toField, null, subQuery);
+  }
+  
 }
 
 
 class JoinQuery extends Query {
   String fromField;
   String toField;
-  String fromIndex;
+  String fromIndex; // TODO: name is missleading here compared to JoinQParserPlugin usage - here it must be a core name
   Query q;
   long fromCoreOpenTime;
 
-  public JoinQuery(String fromField, String toField, String fromIndex, Query subQuery) {
+  public JoinQuery(String fromField, String toField, String coreName, Query subQuery) {
+    assert null != fromField;
+    assert null != toField;
+    assert null != subQuery;
+    
     this.fromField = fromField;
     this.toField = toField;
-    this.fromIndex = fromIndex;
     this.q = subQuery;
+    
+    this.fromIndex = coreName; // may be null
   }
 
   public Query getQuery() { return q; }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/15e1c5d3/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java b/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java
index 9f05d8e..d5a4e23 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java
@@ -161,6 +161,8 @@ public abstract class FacetProcessor<FacetRequestT extends FacetRequest>  {
     // We still calculate them first because we can use it in a parent->child domain change.
     evalFilters();
 
+    handleJoinField();
+    
     boolean appliedFilters = handleBlockJoin();
 
     if (this.filter != null && !appliedFilters) {
@@ -229,6 +231,14 @@ public abstract class FacetProcessor<FacetRequestT extends FacetRequest>  {
     fcontext.base = fcontext.searcher.getDocSet(qlist);
   }
 
+  /** modifies the context base if there is a join field domain change */
+  private void handleJoinField() throws IOException {
+    if (null == freq.domain.joinField) return;
+
+    final Query domainQuery = freq.domain.joinField.createDomainQuery(fcontext);
+    fcontext.base = fcontext.searcher.getDocSet(domainQuery);
+  }
+    
   // returns "true" if filters were applied to fcontext.base already
   private boolean handleBlockJoin() throws IOException {
     boolean appliedFilters = false;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/15e1c5d3/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java b/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
index b446ece..4cf8a68 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.solr.search.facet;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.LinkedHashMap;
@@ -29,10 +30,12 @@ import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.search.DocSet;
+import org.apache.solr.search.JoinQParserPlugin;
 import org.apache.solr.search.FunctionQParser;
 import org.apache.solr.search.FunctionQParserPlugin;
 import org.apache.solr.search.QParser;
 import org.apache.solr.search.QueryContext;
+import org.apache.solr.search.SolrConstantScoreQuery;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.search.SyntaxError;
 
@@ -85,6 +88,7 @@ public abstract class FacetRequest {
   // domain changes
   public static class Domain {
     public List<String> excludeTags;
+    public JoinField joinField;
     public boolean toParent;
     public boolean toChildren;
     public String parents; // identifies the parent filter... the full set of parent documents for any block join operation
@@ -92,13 +96,79 @@ public abstract class FacetRequest {
 
     // True if a starting set of documents can be mapped onto a different set of documents not originally in the starting set.
     public boolean canTransformDomain() {
-      return toParent || toChildren || excludeTags != null;
+      return toParent || toChildren || (excludeTags != null) || (joinField != null);
     }
 
     // Can this domain become non-empty if the input domain is empty?  This does not check any sub-facets (see canProduceFromEmpty for that)
     public boolean canBecomeNonEmpty() {
       return excludeTags != null;
     }
+
+    /** Are we doing a query time join across other documents */
+    public static class JoinField {
+      public final String from;
+      public final String to;
+      
+      private JoinField(String from, String to) {
+        assert null != from;
+        assert null != to;
+        
+        this.from = from;
+        this.to = to;
+      }
+
+      /**
+       * Given a <code>Domain</code>, and a (JSON) map specifying the configuration for that Domain,
+       * validates if a '<code>join</code>' is specified, and if so creates a <code>JoinField</code> 
+       * and sets it on the <code>Domain</code>.
+       *
+       * (params must not be null)
+       */
+      public static void createJoinField(FacetRequest.Domain domain, Map<String,Object> domainMap) {
+        assert null != domain;
+        assert null != domainMap;
+        
+        final Object queryJoin = domainMap.get("join");
+        if (null != queryJoin) {
+          // TODO: maybe allow simple string (instead of map) to mean "self join on this field name" ?
+          if (! (queryJoin instanceof Map)) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+                                    "'join' domain change requires a map containing the 'from' and 'to' fields");
+          }
+          final Map<String,String> join = (Map<String,String>) queryJoin;
+          if (! (join.containsKey("from") && join.containsKey("to") && 
+                 null != join.get("from") && null != join.get("to")) ) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+                                    "'join' domain change requires non-null 'from' and 'to' field names");
+          }
+          if (2 != join.size()) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+                                    "'join' domain change contains unexpected keys, only 'from' and 'to' supported: "
+                                    + join.toString());
+          }
+          domain.joinField = new JoinField(join.get("from"), join.get("to"));
+        }
+      }
+
+      /**
+       * Creates a Query that can be used to recompute the new "base" for this domain, realtive to the 
+       * current base of the FacetContext.
+       */
+      public Query createDomainQuery(FacetContext fcontext) throws IOException {
+        // NOTE: this code lives here, instead of in FacetProcessor.handleJoin, in order to minimize
+        // the number of classes that have to know about the number of possible settings on the join
+        // (ie: if we add a score mode, or some other modifier to how the joins are done)
+        
+        final SolrConstantScoreQuery fromQuery = new SolrConstantScoreQuery(fcontext.base.getTopFilter());
+        // this shouldn't matter once we're wrapped in a join query, but just in case it ever does...
+        fromQuery.setCache(false); 
+
+        return JoinQParserPlugin.createJoinQuery(fromQuery, this.from, this.to);
+      }
+      
+      
+    }
+    
   }
 
   public FacetRequest() {
@@ -399,6 +469,8 @@ abstract class FacetParser<FacetRequestT extends FacetRequest> {
           domain.toChildren = true;
           domain.parents = blockChildren;
         }
+          
+        FacetRequest.Domain.JoinField.createJoinField(domain, domainMap);
 
         Object filterOrList = domainMap.get("filter");
         if (filterOrList != null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/15e1c5d3/solr/core/src/test/org/apache/solr/cloud/TestCloudJSONFacetJoinDomain.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCloudJSONFacetJoinDomain.java b/solr/core/src/test/org/apache/solr/cloud/TestCloudJSONFacetJoinDomain.java
new file mode 100644
index 0000000..afdbb7e
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCloudJSONFacetJoinDomain.java
@@ -0,0 +1,596 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+
+import java.lang.invoke.MethodHandles;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicInteger;
+  
+import org.apache.commons.lang.StringUtils;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.SolrTestCaseJ4.SuppressPointFields;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** 
+ * <p>
+ * Tests randomized JSON Facets, sometimes using query 'join' domain transfers and/or domain 'filter' options
+ * </p>
+ * <p>
+ * The results of each facet constraint count will be compared with a verification query using an equivilent filter
+ * </p>
+ * 
+ * @see TestCloudPivotFacet
+ */
+@SuppressPointFields(bugUrl="https://issues.apache.org/jira/browse/SOLR-9989")
+public class TestCloudJSONFacetJoinDomain extends SolrCloudTestCase {
+  
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final String DEBUG_LABEL = MethodHandles.lookup().lookupClass().getName();
+  private static final String COLLECTION_NAME = DEBUG_LABEL + "_collection";
+
+  private static final int MAX_FIELD_NUM = 15;
+  private static final int UNIQUE_FIELD_VALS = 20;
+  private static final int FACET_LIMIT = UNIQUE_FIELD_VALS + 1;
+  
+  /** Multivalued string field suffixes that can be randomized for testing diff facet/join code paths */
+  private static final String[] STR_FIELD_SUFFIXES = new String[] { "_ss", "_sds", "_sdsS" };
+  /** Multivalued int field suffixes that can be randomized for testing diff facet/join code paths */
+  private static final String[] INT_FIELD_SUFFIXES = new String[] { "_is", "_ids", "_idsS" };
+  
+  /** A basic client for operations at the cloud level, default collection will be set */
+  private static CloudSolrClient CLOUD_CLIENT;
+  /** One client per node */
+  private static ArrayList<HttpSolrClient> CLIENTS = new ArrayList<>(5);
+
+  
+  @BeforeClass
+  private static void createMiniSolrCloudCluster() throws Exception {
+    // sanity check constants
+    assertTrue("bad test constants: must have UNIQUE_FIELD_VALS < FACET_LIMIT since refinement not currently supported",
+               UNIQUE_FIELD_VALS < FACET_LIMIT);
+    assertTrue("bad test constants: some suffixes will never be tested",
+               (STR_FIELD_SUFFIXES.length < MAX_FIELD_NUM) && (INT_FIELD_SUFFIXES.length < MAX_FIELD_NUM));
+    
+    // multi replicas should not matter...
+    final int repFactor = usually() ? 1 : 2;
+    // ... but we definitely want to test multiple shards
+    final int numShards = TestUtil.nextInt(random(), 1, (usually() ? 2 :3));
+    final int numNodes = (numShards * repFactor);
+   
+    final String configName = DEBUG_LABEL + "_config-set";
+    final Path configDir = Paths.get(TEST_HOME(), "collection1", "conf");
+    
+    configureCluster(numNodes).addConfig(configName, configDir).configure();
+    
+    Map<String, String> collectionProperties = new LinkedHashMap<>();
+    collectionProperties.put("config", "solrconfig-tlog.xml");
+    collectionProperties.put("schema", "schema_latest.xml");
+    CollectionAdminRequest.createCollection(COLLECTION_NAME, configName, numShards, repFactor)
+        .setProperties(collectionProperties)
+        .process(cluster.getSolrClient());
+
+    CLOUD_CLIENT = cluster.getSolrClient();
+    CLOUD_CLIENT.setDefaultCollection(COLLECTION_NAME);
+
+    waitForRecoveriesToFinish(CLOUD_CLIENT);
+
+    for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
+      CLIENTS.add(getHttpSolrClient(jetty.getBaseUrl() + "/" + COLLECTION_NAME + "/"));
+    }
+
+    final int numDocs = atLeast(100);
+    for (int id = 0; id < numDocs; id++) {
+      SolrInputDocument doc = sdoc("id", ""+id);
+      for (int fieldNum = 0; fieldNum < MAX_FIELD_NUM; fieldNum++) {
+        // NOTE: some docs may have no value in a field
+        final int numValsThisDoc = TestUtil.nextInt(random(), 0, (usually() ? 3 : 6));
+        for (int v = 0; v < numValsThisDoc; v++) {
+          final String fieldValue = randFieldValue(fieldNum);
+          
+          // for each fieldNum, there are actaully two fields: one string, and one integer
+          doc.addField(field(STR_FIELD_SUFFIXES, fieldNum), fieldValue);
+          doc.addField(field(INT_FIELD_SUFFIXES, fieldNum), fieldValue);
+        }
+      }
+      CLOUD_CLIENT.add(doc);
+    }
+    CLOUD_CLIENT.commit();
+  }
+
+  /**
+   * Given a (random) number, and a (static) array of possible suffixes returns a consistent field name that 
+   * uses that number and one of hte specified suffixes in it's name.
+   *
+   * @see #STR_FIELD_SUFFIXES
+   * @see #INT_FIELD_SUFFIXES
+   * @see #MAX_FIELD_NUM
+   * @see #randFieldValue
+   */
+  private static String field(final String[] suffixes, final int fieldNum) {
+    assert fieldNum < MAX_FIELD_NUM;
+    
+    final String suffix = suffixes[fieldNum % suffixes.length];
+    return "field_" + fieldNum + suffix;
+  }
+  private static String strfield(final int fieldNum) {
+    return field(STR_FIELD_SUFFIXES, fieldNum);
+  }
+  private static String intfield(final int fieldNum) {
+    return field(INT_FIELD_SUFFIXES, fieldNum);
+  }
+
+  /**
+   * Given a (random) field number, returns a random (integer based) value for that field.
+   * NOTE: The number of unique values in each field is constant acording to {@link #UNIQUE_FIELD_VALS}
+   * but the cise pr<em>range</em> of values will vary for each unique field number, such that cross field joins 
+   * will match fewer documents based on how far apart the field numbers are.
+   *
+   * @see #UNIQUE_FIELD_VALS
+   * @see #field
+   */
+  private static String randFieldValue(final int fieldNum) {
+    return "" + (fieldNum + TestUtil.nextInt(random(), 0, UNIQUE_FIELD_VALS));
+  }
+
+  
+  @AfterClass
+  private static void afterClass() throws Exception {
+    CLOUD_CLIENT.close(); CLOUD_CLIENT = null;
+    for (HttpSolrClient client : CLIENTS) {
+      client.close();
+    }
+    CLIENTS = null;
+  }
+
+  /** Sanity check that malformed requests produce errors */
+  public void testMalformedGivesError() throws Exception {
+
+    ignoreException(".*'join' domain change.*");
+    
+    for (String join : Arrays.asList("bogus",
+                                     "{ }",
+                                     "{ from:null, to:foo_s }",
+                                     "{ from:foo_s }",
+                                     "{ from:foo_s, to:foo_s, bogus:'what what?' }",
+                                     "{ to:foo_s, bogus:'what what?' }")) {
+      SolrException e = expectThrows(SolrException.class, () -> {
+          final SolrParams req = params("q", "*:*", "json.facet",
+                                        "{ x : { type:terms, field:x_s, domain: { join:"+join+" } } }");
+          final NamedList trash = getRandClient(random()).request(new QueryRequest(req));
+        });
+      assertEquals(join + " -> " + e, SolrException.ErrorCode.BAD_REQUEST.code, e.code());
+      assertTrue(join + " -> " + e, e.getMessage().contains("'join' domain change"));
+    }
+  }
+
+  public void testSanityCheckDomainMethods() throws Exception {
+    { 
+      final JoinDomain empty = new JoinDomain(null, null, null);
+      assertEquals(null, empty.toJSONFacetParamValue());
+      final SolrParams out = empty.applyDomainToQuery("safe_key", params("q","qqq"));
+      assertNotNull(out);
+      assertEquals(null, out.get("safe_key"));
+      assertEquals("qqq", out.get("q"));
+    }
+    {
+      final JoinDomain join = new JoinDomain("xxx", "yyy", null);
+      assertEquals("domain:{join:{from:xxx,to:yyy}}", join.toJSONFacetParamValue().toString());
+      final SolrParams out = join.applyDomainToQuery("safe_key", params("q","qqq"));
+      assertNotNull(out);
+      assertEquals("qqq", out.get("safe_key"));
+      assertEquals("{!join from=xxx to=yyy v=$safe_key}", out.get("q"));
+      
+    }
+    {
+      final JoinDomain filter = new JoinDomain(null, null, "zzz");
+      assertEquals("domain:{filter:'zzz'}", filter.toJSONFacetParamValue().toString());
+      final SolrParams out = filter.applyDomainToQuery("safe_key", params("q","qqq"));
+      assertNotNull(out);
+      assertEquals(null, out.get("safe_key"));
+      assertEquals("zzz AND qqq", out.get("q"));
+    }
+    {
+      final JoinDomain both = new JoinDomain("xxx", "yyy", "zzz");
+      assertEquals("domain:{join:{from:xxx,to:yyy},filter:'zzz'}", both.toJSONFacetParamValue().toString());
+      final SolrParams out = both.applyDomainToQuery("safe_key", params("q","qqq"));
+      assertNotNull(out);
+      assertEquals("qqq", out.get("safe_key"));
+      assertEquals("zzz AND {!join from=xxx to=yyy v=$safe_key}", out.get("q"));
+    }
+  }
+
+  
+  /** 
+   * Test some small, hand crafted, but non-trivial queries that are
+   * easier to trace/debug then a pure random monstrosity.
+   * (ie: if something obvious gets broken, this test may fail faster and in a more obvious way then testRandom)
+   */
+  public void testBespoke() throws Exception {
+
+    { // strings
+      Map<String,TermFacet> facets = new LinkedHashMap<>();
+      TermFacet top = new TermFacet(strfield(9), new JoinDomain(strfield(5), strfield(9), strfield(9)+":[* TO *]"));
+      top.subFacets.put("facet_5", new TermFacet(strfield(11), new JoinDomain(strfield(8), strfield(8), null)));
+      facets.put("facet_4", top);
+      assertFacetCountsAreCorrect(facets, "("+strfield(7)+":6 OR "+strfield(9)+":6 OR "+strfield(6)+":19 OR "+strfield(0)+":11)");
+    }
+
+    { // ints
+      Map<String,TermFacet> facets = new LinkedHashMap<>();
+      TermFacet top = new TermFacet(intfield(9), new JoinDomain(intfield(5), intfield(9), null));
+      facets.put("top", top);
+      assertFacetCountsAreCorrect(facets, "("+intfield(7)+":6 OR "+intfield(3)+":3)");
+    }
+
+    { // some domains with filter only, no actual join
+      Map<String,TermFacet> facets = new LinkedHashMap<>();
+      TermFacet top = new TermFacet(strfield(9), new JoinDomain(null, null, strfield(9)+":[* TO *]"));
+      top.subFacets.put("facet_5", new TermFacet(strfield(11), new JoinDomain(null, null, strfield(3)+":[* TO 5]")));
+      facets.put("top", top);
+      assertFacetCountsAreCorrect(facets, "("+strfield(7)+":6 OR "+strfield(9)+":6 OR "+strfield(6)+":19 OR "+strfield(0)+":11)");
+
+    }
+
+    
+  }
+
+  
+  public void testRandom() throws Exception {
+
+    final int numIters = atLeast(3);
+    for (int iter = 0; iter < numIters; iter++) {
+      assertFacetCountsAreCorrect(TermFacet.buildRandomFacets(), buildRandomQuery());
+    }
+  }
+
+  /**
+   * Generates a random query string across the randomized fields/values in the index
+   *
+   * @see #randFieldValue
+   * @see #field
+   */
+  private static String buildRandomQuery() {
+    if (0 == TestUtil.nextInt(random(), 0,10)) {
+      return "*:*";
+    }
+    final int numClauses = TestUtil.nextInt(random(), 3, 10);
+    List<String> clauses = new ArrayList<String>(numClauses);
+    for (int c = 0; c < numClauses; c++) {
+      final int fieldNum = random().nextInt(MAX_FIELD_NUM);
+      // keep queries simple, just use str fields - not point of test
+      clauses.add(strfield(fieldNum) + ":" + randFieldValue(fieldNum));
+    }
+    return "(" + StringUtils.join(clauses, " OR ") + ")";
+  }
+  
+  /**
+   * Given a set of (potentially nested) term facets, and a base query string, asserts that 
+   * the actual counts returned when executing that query with those facets match the expected results
+   * of filtering on the equivilent facet terms+domain
+   */
+  private void assertFacetCountsAreCorrect(Map<String,TermFacet> expected,
+                                           final String query) throws SolrServerException, IOException {
+
+    final SolrParams baseParams = params("q", query, "rows","0");
+    final SolrParams facetParams = params("json.facet", ""+TermFacet.toJSONFacetParamValue(expected));
+    final SolrParams initParams = SolrParams.wrapAppended(facetParams, baseParams);
+    
+    log.info("Doing full run: {}", initParams);
+    
+    NamedList topResponse = null;
+    try {
+      topResponse = getRandClient(random()).request(new QueryRequest(initParams));
+      assertNotNull(initParams + " is null response?", topResponse);
+    } catch (Exception e) {
+      throw new RuntimeException("init query failed: " + initParams + ": " + 
+                                 e.getMessage(), e);
+    }
+    try {
+      final NamedList facetResponse = (NamedList) topResponse.get("facets");
+      assertNotNull("null facet results?", facetResponse);
+      assertFacetCountsAreCorrect(expected, baseParams, facetResponse);
+    } catch (AssertionError e) {
+      throw new AssertionError(initParams + " ===> " + topResponse + " --> " + e.getMessage(), e);
+    } finally {
+      log.info("Ending full run"); 
+    }
+  }
+
+  /** 
+   * Recursive Helper method that walks the actual facet response, comparing the counts to the expected output 
+   * based on the equivilent filters generated from the original TermFacet.
+   */
+  private void assertFacetCountsAreCorrect(Map<String,TermFacet> expected,
+                                           SolrParams baseParams,
+                                           NamedList actualFacetResponse) throws SolrServerException, IOException {
+
+    for (Map.Entry<String,TermFacet> entry : expected.entrySet()) {
+      final String facetKey = entry.getKey();
+      final TermFacet facet = entry.getValue();
+      final NamedList results = (NamedList) actualFacetResponse.get(facetKey);
+      assertNotNull(facetKey + " key missing from: " + actualFacetResponse, results);
+      final List<NamedList> buckets = (List<NamedList>) results.get("buckets");
+      assertNotNull(facetKey + " has null buckets: " + actualFacetResponse, buckets);
+      for (NamedList bucket : buckets) {
+        final long count = ((Number) bucket.get("count")).longValue();
+        final String fieldVal = bucket.get("val").toString(); // int or stringified int
+
+        // change our query to filter on the fieldVal, and wrap in the facet domain (if any)
+        final SolrParams verifyParams = facet.applyValueConstraintAndDomain(baseParams, facetKey, fieldVal);
+
+        // check the count for this bucket
+        assertEquals(facetKey + ": " + verifyParams,
+                     count, getRandClient(random()).query(verifyParams).getResults().getNumFound());
+
+        // recursively check subFacets
+        if (! facet.subFacets.isEmpty()) {
+          assertFacetCountsAreCorrect(facet.subFacets,
+                                      verifyParams, bucket);
+        }
+      }
+    }
+    assertTrue("facets have unexpeted keys left over: " + actualFacetResponse,
+               // should alwasy be a count, maybe a 'val' if we're a subfacet
+               (actualFacetResponse.size() == expected.size() + 1) ||
+               (actualFacetResponse.size() == expected.size() + 2));
+  }
+
+  
+  /**
+   * Trivial data structure for modeling a simple terms facet that can be written out as a json.facet param.
+   *
+   * Doesn't do any string escaping or quoting, so don't use whitespace or reserved json characters
+   */
+  private static final class TermFacet {
+    public final String field;
+    public final Map<String,TermFacet> subFacets = new LinkedHashMap<>();
+    public final JoinDomain domain; // may be null
+    public TermFacet(String field, JoinDomain domain) {
+      assert null != field;
+      this.field = field;
+      this.domain = domain;
+    }
+
+    /** 
+     * Returns new SolrParams that:
+     * <ul>
+     *  <li>copy the original SolrParams</li>
+     *  <li>modify/wrap the original "q" param to capture the domain change for this facet (if any)</li>
+     *  <li>add a filter query against this field with the specified value</li>
+     * </ul>
+     * 
+     * @see JoinDomain#applyDomainToQuery
+     */
+    public SolrParams applyValueConstraintAndDomain(SolrParams orig, String facetKey, String facetVal) {
+      // first wrap our original query in the domain if there is one...
+      if (null != domain) {
+        orig = domain.applyDomainToQuery(facetKey + "_q", orig);
+      }
+      // then filter by the facet value we need to test...
+      final ModifiableSolrParams out = new ModifiableSolrParams(orig);
+      out.set("q", field + ":" + facetVal + " AND " + orig.get("q"));
+
+      return out;
+    }
+    
+    /**
+     * recursively generates the <code>json.facet</code> param value to use for testing this facet
+     */
+    private CharSequence toJSONFacetParamValue() {
+      // NOTE: since refinement isn't supported, we have to use the max cardinality of the field as limit
+      StringBuilder sb = new StringBuilder("{ type:terms, field:" + field + ", limit: " + FACET_LIMIT);
+      if (! subFacets.isEmpty()) {
+        sb.append(", facet:");
+        sb.append(toJSONFacetParamValue(subFacets));
+      }
+      if (null != domain) {
+        CharSequence ds = domain.toJSONFacetParamValue();
+        if (null != ds) {
+          sb.append(", ").append(ds);
+        }
+      }
+      sb.append("}");
+      return sb;
+    }
+    
+    /**
+     * Given a set of (possibly nested) facets, generates a suitable <code>json.facet</code> param value to 
+     * use for testing them against in a solr request.
+     */
+    public static CharSequence toJSONFacetParamValue(Map<String,TermFacet> facets) {
+      assert null != facets;
+      assert 0 < facets.size();
+      StringBuilder sb = new StringBuilder("{");
+      for (String key : facets.keySet()) {
+        sb.append(key).append(" : ").append(facets.get(key).toJSONFacetParamValue());
+        sb.append(" ,");
+      }
+      sb.setLength(sb.length() - 1);
+      sb.append("}");
+      return sb;
+    }
+    
+    /**
+     * Factory method for generating some random (nested) facets.  
+     *
+     * For simplicity, each facet will have a unique key name, regardless of it's depth under other facets 
+     *
+     * @see JoinDomain
+     */
+    public static Map<String,TermFacet> buildRandomFacets() {
+      // for simplicity, use a unique facet key regardless of depth - simplifies verification
+      AtomicInteger keyCounter = new AtomicInteger(0);
+      final int maxDepth = TestUtil.nextInt(random(), 0, (usually() ? 2 : 3));
+      return buildRandomFacets(keyCounter, maxDepth);
+    }
+
+    /** 
+     * recursive helper method for building random facets
+     *
+     * @param keyCounter used to ensure every generated facet has a unique key name
+     * @param maxDepth max possible depth allowed for the recusion, a lower value may be used depending on how many facets are returned at the current level. 
+     */
+    private static Map<String,TermFacet> buildRandomFacets(AtomicInteger keyCounter, int maxDepth) {
+      final int numFacets = Math.max(1, TestUtil.nextInt(random(), -1, 3)); // 3/5th chance of being '1'
+      Map<String,TermFacet> results = new LinkedHashMap<>();
+      for (int i = 0; i < numFacets; i++) {
+        final JoinDomain domain = JoinDomain.buildRandomDomain();
+        assert null != domain;
+        final TermFacet facet =  new TermFacet(field(random().nextBoolean() ? STR_FIELD_SUFFIXES : INT_FIELD_SUFFIXES,
+                                                     random().nextInt(MAX_FIELD_NUM)),
+                                               domain);
+        results.put("facet_" + keyCounter.incrementAndGet(), facet);
+        if (0 < maxDepth) {
+          // if we're going wide, don't go deep
+          final int nextMaxDepth = Math.max(0, maxDepth - numFacets);
+          facet.subFacets.putAll(buildRandomFacets(keyCounter, TestUtil.nextInt(random(), 0, nextMaxDepth)));
+        }
+      }
+      return results;
+    }
+  }
+
+
+  /**
+   * Models a Domain Change which includes either a 'join' or a 'filter' or both
+   */
+  private static final class JoinDomain { 
+    public final String from;
+    public final String to;
+    public final String filter; // not bothering with more then 1 filter, not the point of the test
+
+    /** 
+     * @param from left side of join field name, null if domain involves no joining
+     * @param to right side of join field name, null if domain involves no joining
+     * @param filter filter to apply to domain, null if domain involves no filtering
+     */
+    public JoinDomain(String from, String to, String filter) { 
+      assert ! ((null ==  from) ^ (null == to)) : "if from is null, to must be null";
+      this.from = from;
+      this.to = to;
+      this.filter = filter;
+    }
+
+    /** 
+     * @return the JSON string representing this domain for use in a facet param, or null if no domain should be used
+     * */
+    public CharSequence toJSONFacetParamValue() {
+      if (null == from && null == filter) {
+        return null;
+      }
+      StringBuilder sb = new StringBuilder("domain:{");
+      if (null != from) {
+        assert null != to;
+        sb. append("join:{from:").append(from).append(",to:").append(to).append("}");
+        if (null != filter){
+          sb.append(",");
+        }
+        
+      }
+      if (null != filter) {
+        sb.append("filter:'").append(filter).append("'");
+      }
+      sb.append("}");
+      return sb;
+    }
+
+    /** 
+     * Given some original SolrParams, returns new SolrParams where the original "q" param is wrapped
+     * as needed to apply the equivilent transformation to a query as this domain would to a facet
+     */
+    public SolrParams applyDomainToQuery(String safeKey, SolrParams in) {
+      assert null == in.get(safeKey); // shouldn't be possible if every facet uses a unique key string
+      
+      String q = in.get("q");
+      final ModifiableSolrParams out = new ModifiableSolrParams(in);
+      if (null != from) {
+        out.set(safeKey, in.get("q"));
+        q =  "{!join from="+from+" to="+to+" v=$"+safeKey+"}";
+      }
+      if (null != filter) {
+        q = filter + " AND " + q;
+      }
+      out.set("q", q);
+      return out;
+    }
+
+    /**
+     * Factory method for creating a random domain change to use with a facet - may return an 'noop' JoinDomain,
+     * but will never return null.
+     */
+    public static JoinDomain buildRandomDomain() { 
+
+      // use consistent type on both sides of join
+      final String[] suffixes = random().nextBoolean() ? STR_FIELD_SUFFIXES : INT_FIELD_SUFFIXES;
+      
+      final boolean noJoin = random().nextBoolean();
+      final String from = noJoin ? null : field(suffixes, random().nextInt(MAX_FIELD_NUM));
+      final String to = noJoin ? null : field(suffixes, random().nextInt(MAX_FIELD_NUM));
+      
+      // keep it simple, only filter on string fields - not point of test
+      final String filterField = strfield(random().nextInt(MAX_FIELD_NUM));
+      
+      final String filter = random().nextBoolean() ? null : filterField+":[* TO *]";
+      return new JoinDomain(from, to, filter);
+    }
+  }
+  
+  /** 
+   * returns a random SolrClient -- either a CloudSolrClient, or an HttpSolrClient pointed 
+   * at a node in our cluster 
+   */
+  public static SolrClient getRandClient(Random rand) {
+    int numClients = CLIENTS.size();
+    int idx = TestUtil.nextInt(rand, 0, numClients);
+
+    return (idx == numClients) ? CLOUD_CLIENT : CLIENTS.get(idx);
+  }
+
+  public static void waitForRecoveriesToFinish(CloudSolrClient client) throws Exception {
+    assert null != client.getDefaultCollection();
+    AbstractDistribZkTestBase.waitForRecoveriesToFinish(client.getDefaultCollection(),
+                                                        client.getZkStateReader(),
+                                                        true, true, 330);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/15e1c5d3/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java
index 6353576..6b542a1 100644
--- a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java
+++ b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java
@@ -416,8 +416,54 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS {
             "}"
     );
 
+    final String sort_limit_over = "sort:'count desc', limit:1, overrequest:0, ";
+    // simplistic join domain testing: no refinement == low count
+    client.testJQ(params(p, "q", "${xy_s}:Y", // query only matches one doc per shard
+                         "json.facet", "{"+
+                         "  cat0:{type:terms, field:${cat_s}, "+sort_limit_over+" refine:false,"+
+                         // self join on all_s ensures every doc on every shard included in facets
+                         "        domain: { join: { from:all_s, to:all_s } } }" +
+                         "}"
+                         )
+                  ,
+                  "/response/numFound==3",
+                  "facets=={ count:3, " +
+                  // w/o overrequest and refinement, count for 'A' is lower than it should be
+                  // (we don't see the A from the middle shard)
+                  "          cat0:{ buckets:[ {val:A,count:3} ] } }");
+    // simplistic join domain testing: refinement == correct count
+    client.testJQ(params(p, "q", "${xy_s}:Y", // query only matches one doc per shard
+                         "json.facet", "{" +
+                         "  cat0:{type:terms, field:${cat_s}, "+sort_limit_over+" refine:true,"+
+                         // self join on all_s ensures every doc on every shard included in facets
+                         "        domain: { join: { from:all_s, to:all_s } } }" +
+                         "}"
+                         )
+                  ,
+                  "/response/numFound==3",
+                  "facets=={ count:3," +
+                  // w/o overrequest, we need refining to get the correct count for 'A'.
+                  "          cat0:{ buckets:[ {val:A,count:4} ] } }");
+
+    // contrived join domain + refinement (at second level) + testing
+    client.testJQ(params(p, "q", "${xy_s}:Y", // query only matches one doc per shard
+                         "json.facet", "{" +
+                         // top level facet has a single term
+                         "  all:{type:terms, field:all_s, "+sort_limit_over+" refine:true, " +
+                         "       facet:{  "+
+                         // subfacet will facet on cat after joining on all (so all docs should be included in subfacet)
+                         "         cat0:{type:terms, field:${cat_s}, "+sort_limit_over+" refine:true,"+
+                         "               domain: { join: { from:all_s, to:all_s } } } } }" +
+                         "}"
+                         )
+                  ,
+                  "/response/numFound==3",
+                  "facets=={ count:3," +
+                  // all 3 docs matching base query have same 'all' value in top facet
+                  "          all:{ buckets:[ { val:all, count:3, " +
+                  // sub facet has refinement, so count for 'A' should be correct
+                  "                            cat0:{ buckets: [{val:A,count:4}] } } ] } }");
 
   }
-
-
+  
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/15e1c5d3/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
index a2b6f06..3f31cca 100644
--- a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
+++ b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
@@ -37,6 +37,7 @@ import org.apache.solr.SolrTestCaseHS;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.SolrTestCaseJ4.SuppressPointFields;
 import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.request.macro.MacroExpander;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -205,6 +206,108 @@ public class TestJsonFacets extends SolrTestCaseHS {
     client.commit();
   }
 
+  public void testDomainJoinSelf() throws Exception {
+    Client client = Client.localClient();
+    indexSimple(client);
+
+    // self join domain switch at the second level of faceting
+    assertJQ(req("q", "*:*", "rows", "0",
+                 "json.facet", ""
+                 + "{x: { type: terms, field: 'num_i', "
+                 + "      facet: { y: { domain: { join: { from: 'cat_s', to: 'cat_s' } }, "
+                 + "                    type: terms, field: 'where_s' "
+                 + "                  } } } }")
+             , "facets=={count:6, x:{ buckets:["
+             + "   { val:-5, count:2, "
+             + "     y : { buckets:[{ val:'NJ', count:2 }, { val:'NY', count:1 } ] } }, "
+             + "   { val:2, count:1, "
+             + "     y : { buckets:[{ val:'NJ', count:1 }, { val:'NY', count:1 } ] } }, "
+             + "   { val:3, count:1, "
+             + "     y : { buckets:[{ val:'NJ', count:1 }, { val:'NY', count:1 } ] } }, "
+             + "   { val:7, count:1, "
+             + "     y : { buckets:[{ val:'NJ', count:2 }, { val:'NY', count:1 } ] } } ] } }"
+             );
+  }
+  
+  public void testNestedJoinDomain() throws Exception {
+    Client client = Client.localClient();
+
+    client.deleteByQuery("*:*", null);
+    client.add(sdoc("id", "1", "1_s", "A", "2_s", "A", "3_s", "C", "y_s", "B", "x_t", "x   z", "z_t", "  2 3"), null);
+    client.add(sdoc("id", "2", "1_s", "B", "2_s", "A", "3_s", "B", "y_s", "B", "x_t", "x y  ", "z_t", "1   3"), null);
+    client.add(sdoc("id", "3", "1_s", "C", "2_s", "A", "3_s", "#", "y_s", "A", "x_t", "  y z", "z_t", "1 2  "), null);
+    client.add(sdoc("id", "4", "1_s", "A", "2_s", "B", "3_s", "C", "y_s", "A", "x_t", "    z", "z_t", "    3"), null);
+    client.add(sdoc("id", "5", "1_s", "B", "2_s", "_", "3_s", "B", "y_s", "C", "x_t", "x    ", "z_t", "1   3"), null);
+    client.add(sdoc("id", "6", "1_s", "C", "2_s", "B", "3_s", "A", "y_s", "C", "x_t", "x y z", "z_t", "1    "), null);
+    client.commit();
+
+    assertJQ(req("q", "x_t:x", "rows", "0", // NOTE q - only x=x in base set (1,2,5,6)
+                 "json.facet", ""
+                 + "{x: { type: terms, field: 'x_t', "
+                 + "      domain: { join: { from:'1_s', to:'2_s' } },"
+                 //                y1 & y2 are the same facet, with *similar* child facet z1/z2 ...
+                 + "      facet: { y1: { type: terms, field: 'y_s', "
+                 //                               z1 & z2 are same field, diff join...
+                 + "                     facet: { z1: { type: terms, field: 'z_t', "
+                 + "                                    domain: { join: { from:'2_s', to:'3_s' } } } } },"
+                 + "               y2: { type: terms, field: 'y_s', "
+                 //                               z1 & z2 are same field, diff join...
+                 + "                     facet: { z2: { type: terms, field: 'z_t', "
+                 + "                                    domain: { join: { from:'3_s', to:'1_s' } } } } } } } }")
+             , "facets=={count:4, "
+             + "x:{ buckets:[" // joined 1->2: doc5 drops out, counts: z=4, x=3, y=3 
+             + "   { val:z, count:4, " // x=z (docs 1,3,4,6) y terms: A=2, B=1, C=1
+             + "     y1 : { buckets:[ " // z1 joins 2->3...
+             + "             { val:A, count:2, " // A in docs(3,4), joins (A,B) -> docs(2,5,6)
+             + "               z1: { buckets:[{ val:'1', count:3 }, { val:'3', count:2 }] } }, "
+             + "             { val:B, count:1, " // B in doc1, joins A -> doc6
+             + "               z1: { buckets:[{ val:'1', count:1 }] } }, "
+             + "             { val:C, count:1, " // C in doc6, joins B -> docs(2,5)
+             + "               z1: { buckets:[{ val:'1', count:2 }, { val:'3', count:2 }] } } "
+             + "          ] }, "
+             + "     y2 : { buckets:[ " // z2 joins 3->1...
+             + "             { val:A, count:2, " // A in docs(3,4), joins C -> docs(3,6)
+             + "               z2: { buckets:[{ val:'1', count:2 }, { val:'2', count:1 }] } }, "
+             + "             { val:B, count:1, " // B in doc1, joins C -> docs(3,6)
+             + "               z2: { buckets:[{ val:'1', count:2 }, { val:'2', count:1 }] } }, "
+             + "             { val:C, count:1, " // C in doc6, joins A -> docs(1,4)
+             + "               z2: { buckets:[{ val:'3', count:2 }, { val:'2', count:1 }] } } "
+             + "          ] } }, "
+             + "   { val:x, count:3, " // x=x (docs 1,2,!5,6) y terms: B=2, C=1 
+             + "     y1 : { buckets:[ " // z1 joins 2->3...
+             + "             { val:B, count:2, " // B in docs(1,2), joins A -> doc6
+             + "               z1: { buckets:[{ val:'1', count:1 }] } }, "
+             + "             { val:C, count:1, " // C in doc6, joins B -> docs(2,5)
+             + "               z1: { buckets:[{ val:'1', count:2 }, { val:'3', count:2 }] } } "
+             + "          ] }, "
+             + "     y2 : { buckets:[ " // z2 joins 3->1...
+             + "             { val:B, count:2, " // B in docs(1,2), joins C,B -> docs(2,3,5,6)
+             + "               z2: { buckets:[{ val:'1', count:4 }, { val:'3', count:2 }, { val:'2', count:1 }] } }, "
+             + "             { val:C, count:1, " // C in doc6, joins A -> docs(1,4)
+             + "               z2: { buckets:[{ val:'3', count:2 }, { val:'2', count:1 }] } } "
+             + "          ] } }, "
+             + "   { val:y, count:3, " // x=y (docs 2,3,6) y terms: A=1, B=1, C=1 
+             + "     y1 : { buckets:[ " // z1 joins 2->3...
+             + "             { val:A, count:1, " // A in doc3, joins A -> doc6
+             + "               z1: { buckets:[{ val:'1', count:1 }] } }, "
+             + "             { val:B, count:1, " // B in doc2, joins A -> doc6
+             + "               z1: { buckets:[{ val:'1', count:1 }] } }, "
+             + "             { val:C, count:1, " // C in doc6, joins B -> docs(2,5)
+             + "               z1: { buckets:[{ val:'1', count:2 }, { val:'3', count:2 }] } } "
+             + "          ] }, "
+             + "     y2 : { buckets:[ " // z2 joins 3->1...
+             + "             { val:A, count:1, " // A in doc3, joins # -> empty set
+             + "               z2: { buckets:[ ] } }, "
+             + "             { val:B, count:1, " // B in doc2, joins B -> docs(2,5)
+             + "               z2: { buckets:[{ val:'1', count:2 }, { val:'3', count:2 }] } }, "
+             + "             { val:C, count:1, " // C in doc6, joins A -> docs(1,4)
+             + "               z2: { buckets:[{ val:'3', count:2 }, { val:'2', count:1 }] } } "
+             + "          ]}  }"
+             + "   ]}}"
+             );
+  }
+
+  
   @Test
   public void testMethodStream() throws Exception {
     Client client = Client.localClient();
@@ -1547,6 +1650,142 @@ public class TestJsonFacets extends SolrTestCaseHS {
   }
 
 
+  /**
+   * Similar to {@link #testBlockJoin} but uses query time joining.
+   * <p>
+   * (asserts are slightly diff because if a query matches multiple types of documents, blockJoin domain switches
+   * to parent/child domains preserve any existing parent/children from the original domain - eg: when q=*:*)
+   * </p>
+   */
+  public void testQureyJoinBooksAndPages() throws Exception {
+
+    final Client client = Client.localClient();
+
+    final SolrParams p = params("rows","0");
+
+    client.deleteByQuery("*:*", null);
+
+
+    // build up a list of the docs we want to test with
+    List<SolrInputDocument> docsToAdd = new ArrayList<>(10);
+    docsToAdd.add(sdoc("id", "1", "type_s","book", "book_s","A", "v_t","q"));
+    
+    docsToAdd.add( sdoc("id", "2", "type_s","book", "book_s","B", "v_t","q w") );
+    docsToAdd.add( sdoc("book_id_s", "2", "id", "2.1", "type_s","page", "page_s","a", "v_t","x y z") );
+    docsToAdd.add( sdoc("book_id_s", "2", "id", "2.2", "type_s","page", "page_s","b", "v_t","x y  ") );
+    docsToAdd.add( sdoc("book_id_s", "2", "id","2.3", "type_s","page", "page_s","c", "v_t","  y z" ) );
+
+    docsToAdd.add( sdoc("id", "3", "type_s","book", "book_s","C", "v_t","q w e") );
+    docsToAdd.add( sdoc("book_id_s", "3", "id","3.1", "type_s","page", "page_s","d", "v_t","x    ") );
+    docsToAdd.add( sdoc("book_id_s", "3", "id","3.2", "type_s","page", "page_s","e", "v_t","  y  ") );
+    docsToAdd.add( sdoc("book_id_s", "3", "id","3.3", "type_s","page", "page_s","f", "v_t","    z") );
+
+    docsToAdd.add( sdoc("id", "4", "type_s","book", "book_s","D", "v_t","e") );
+    
+    // shuffle the docs since order shouldn't matter
+    Collections.shuffle(docsToAdd, random());
+    for (SolrInputDocument doc : docsToAdd) {
+      client.add(doc, null);
+    }
+    client.commit();
+
+    // the domains we'll be testing, initially setup for block join
+    final String toChildren = "join: { from:'id', to:'book_id_s' }";
+    final String toParents = "join: { from:'book_id_s', to:'id' }";
+    final String toBogusChildren = "join: { from:'id', to:'does_not_exist' }";
+    final String toBogusParents = "join: { from:'book_id_s', to:'does_not_exist' }";
+
+    client.testJQ(params(p, "q", "*:*"
+            , "json.facet", "{ " +
+                "pages:{ type:query, domain:{"+toChildren+"} , facet:{ x:{field:v_t} } }" +
+                ",pages2:{type:terms, field:v_t, domain:{"+toChildren+"} }" +
+                ",books:{ type:query, domain:{"+toParents+"}  , facet:{ x:{field:v_t} } }" +
+                ",books2:{type:terms, field:v_t, domain:{"+toParents+"} }" +
+                ",pageof3:{ type:query, q:'id:3', facet : { x : { type:terms, field:page_s, domain:{"+toChildren+"}}} }" +
+                ",bookof22:{ type:query, q:'id:2.2', facet : { x : { type:terms, field:book_s, domain:{"+toParents+"}}} }" +
+                ",missing_Parents:{ type:query, domain:{"+toBogusParents+"} }" +
+                ",missing_Children:{ type:query, domain:{"+toBogusChildren+"} }" +
+                "}"
+        )
+        , "facets=={ count:10" +
+            ", pages:{count:6 , x:{buckets:[ {val:y,count:4},{val:x,count:3},{val:z,count:3} ]}  }" +
+            ", pages2:{ buckets:[ {val:y,count:4},{val:x,count:3},{val:z,count:3} ] }" +
+            ", books:{count:2 , x:{buckets:[ {val:q,count:2},{val:w,count:2},{val:e,count:1} ]}  }" +
+            ", books2:{ buckets:[ {val:q,count:2},{val:w,count:2},{val:e,count:1} ] }" +
+            ", pageof3:{count:1 , x:{buckets:[ {val:d,count:1},{val:e,count:1},{val:f,count:1} ]}  }" +
+            ", bookof22:{count:1 , x:{buckets:[ {val:B,count:1} ]}  }" +
+            ", missing_Parents:{count:0}" + 
+            ", missing_Children:{count:0}" +
+            "}"
+    );
+
+    // no matches in base query
+    client.testJQ(params("q", "no_match_s:NO_MATCHES"
+            , "json.facet", "{ processEmpty:true," +
+                "pages:{ type:query, domain:{"+toChildren+"} }" +
+                ",books:{ type:query, domain:{"+toParents+"} }" +
+                "}"
+        )
+        , "facets=={ count:0" +
+            ", pages:{count:0}" +
+            ", books:{count:0}" +
+            "}"
+    );
+
+
+    // test facet on children nested under terms facet on parents
+    client.testJQ(params("q", "*:*"
+            , "json.facet", "{" +
+                "books:{ type:terms, field:book_s, facet:{ pages:{type:terms, field:v_t, domain:{"+toChildren+"}} } }" +
+                "}"
+        )
+        , "facets=={ count:10" +
+            ", books:{buckets:[{val:A,count:1,pages:{buckets:[]}}" +
+            "                 ,{val:B,count:1,pages:{buckets:[{val:y,count:3},{val:x,count:2},{val:z,count:2}]}}" +
+            "                 ,{val:C,count:1,pages:{buckets:[{val:x,count:1},{val:y,count:1},{val:z,count:1}]}}" +
+            "                 ,{val:D,count:1,pages:{buckets:[]}}"+
+            "] }" +
+            "}"
+    );
+
+    // test filter after join
+    client.testJQ(params(p, "q", "*:*"
+        , "json.facet", "{ " +
+            "pages1:{type:terms, field:v_t, domain:{"+toChildren+", filter:'*:*'} }" +
+            ",pages2:{type:terms, field:v_t, domain:{"+toChildren+", filter:'-id:3.1'} }" +
+            ",books:{type:terms, field:v_t, domain:{"+toParents+", filter:'*:*'} }" +
+            ",books2:{type:terms, field:v_t, domain:{"+toParents+", filter:'id:2'} }" +
+            "}"
+        )
+        , "facets=={ count:10" +
+            ", pages1:{ buckets:[ {val:y,count:4},{val:x,count:3},{val:z,count:3} ] }" +
+            ", pages2:{ buckets:[ {val:y,count:4},{val:z,count:3},{val:x,count:2} ] }" +
+            ", books:{ buckets:[ {val:q,count:2},{val:w,count:2},{val:e,count:1} ] }" +
+            ", books2:{ buckets:[ {val:q,count:1}, {val:w,count:1} ] }" +
+            "}"
+    );
+
+
+    // test other various ways to get filters
+    client.testJQ(params(p, "q", "*:*", "f1","-id:3.1", "f2","id:2"
+        , "json.facet", "{ " +
+            "pages1:{type:terms, field:v_t, domain:{"+toChildren+", filter:[]} }" +
+            ",pages2:{type:terms, field:v_t, domain:{"+toChildren+", filter:{param:f1} } }" +
+            ",books:{type:terms, field:v_t, domain:{"+toParents+", filter:[{param:q},{param:missing_param}]} }" +
+            ",books2:{type:terms, field:v_t, domain:{"+toParents+", filter:[{param:f2}] } }" +
+            "}"
+        )
+        , "facets=={ count:10" +
+            ", pages1:{ buckets:[ {val:y,count:4},{val:x,count:3},{val:z,count:3} ] }" +
+            ", pages2:{ buckets:[ {val:y,count:4},{val:z,count:3},{val:x,count:2} ] }" +
+            ", books:{ buckets:[ {val:q,count:2},{val:w,count:2},{val:e,count:1} ] }" +
+            ", books2:{ buckets:[ {val:q,count:1}, {val:w,count:1} ] }" +
+            "}"
+    );
+
+  }
+
+
   @Test
   public void testErrors() throws Exception {
     doTestErrors(Client.localClient());