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 2018/03/11 05:29:55 UTC

lucene-solr:master: SOLR-9510: introducting {!filter param=$fq excludeTags=t, q}foo:bar, also adding filters and excludeTags local params into {!parent} and {!child}

Repository: lucene-solr
Updated Branches:
  refs/heads/master e926f435d -> fa55f0c2f


SOLR-9510: introducting {!filter param=$fq excludeTags=t,q}foo:bar, also adding
filters and excludeTags local params into {!parent} and {!child}


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

Branch: refs/heads/master
Commit: fa55f0c2f2c4af8bca2cd53a76ee6c3142d72ce8
Parents: e926f43
Author: Mikhail Khludnev <mk...@apache.org>
Authored: Mon Mar 5 07:50:54 2018 +0300
Committer: Mikhail Khludnev <mk...@apache.org>
Committed: Sun Mar 11 08:25:05 2018 +0300

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../org/apache/solr/search/QParserPlugin.java   |   2 +
 .../org/apache/solr/search/QueryParsing.java    |  10 +-
 .../solr/search/join/BlockJoinChildQParser.java |  17 +
 .../search/join/BlockJoinParentQParser.java     |  39 ++-
 .../apache/solr/search/join/FiltersQParser.java | 184 ++++++++++
 .../solr/search/join/FiltersQParserPlugin.java  |  33 ++
 .../apache/solr/search/QueryEqualityTest.java   |  55 +++
 .../facet/TestJsonFacetsWithNestedObjects.java  | 337 +++++++++++++++++++
 .../apache/solr/search/join/BJQParserTest.java  | 140 +++++++-
 10 files changed, 787 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fa55f0c2/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index f7c8e78..ab9a9c0 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -200,6 +200,9 @@ New Features
 
 * SOLR-12077: Add support for autoAddReplicas in the collection creation dialog in Admin UI. (shalin)
 
+* SOLR-9510: introducing {!filters param=$fq excludeTags=f} query parser. 
+  Introducing {!.. filters=$fq excludeTags=t,q} in {!parent} and {!child} (Dr. Oleg Savrasov via Mikhail Khludnev)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fa55f0c2/solr/core/src/java/org/apache/solr/search/QParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/QParserPlugin.java b/solr/core/src/java/org/apache/solr/search/QParserPlugin.java
index 893783d..f80bc9c 100644
--- a/solr/core/src/java/org/apache/solr/search/QParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/QParserPlugin.java
@@ -27,6 +27,7 @@ import org.apache.solr.core.SolrInfoBean;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.search.join.BlockJoinChildQParserPlugin;
 import org.apache.solr.search.join.BlockJoinParentQParserPlugin;
+import org.apache.solr.search.join.FiltersQParserPlugin;
 import org.apache.solr.search.join.GraphQParserPlugin;
 import org.apache.solr.search.mlt.MLTQParserPlugin;
 import org.apache.solr.util.plugin.NamedListInitializedPlugin;
@@ -66,6 +67,7 @@ public abstract class QParserPlugin implements NamedListInitializedPlugin, SolrI
     map.put(MaxScoreQParserPlugin.NAME, MaxScoreQParserPlugin.class);
     map.put(BlockJoinParentQParserPlugin.NAME, BlockJoinParentQParserPlugin.class);
     map.put(BlockJoinChildQParserPlugin.NAME, BlockJoinChildQParserPlugin.class);
+    map.put(FiltersQParserPlugin.NAME, FiltersQParserPlugin.class);
     map.put(CollapsingQParserPlugin.NAME, CollapsingQParserPlugin.class);
     map.put(SimpleQParserPlugin.NAME, SimpleQParserPlugin.class);
     map.put(ComplexPhraseQParserPlugin.NAME, ComplexPhraseQParserPlugin.class);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fa55f0c2/solr/core/src/java/org/apache/solr/search/QueryParsing.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/QueryParsing.java b/solr/core/src/java/org/apache/solr/search/QueryParsing.java
index f636aca..c7cd49c 100644
--- a/solr/core/src/java/org/apache/solr/search/QueryParsing.java
+++ b/solr/core/src/java/org/apache/solr/search/QueryParsing.java
@@ -97,12 +97,12 @@ public class QueryParsing {
         throw new SyntaxError("Expected ending character '" + endChar + "' parsing local params '" + txt + '"');
 
       }
-      String val = null;
+      String[] val = new String[1];
 
       ch = p.peek();
       if (ch != '=') {
         // single word... treat {!func} as type=func for easy lookup
-        val = id;
+        val[0] = id;
         id = TYPE;
       } else {
         // saw equals, so read value
@@ -116,7 +116,7 @@ public class QueryParsing {
         }
 
         if (ch == '\"' || ch == '\'') {
-          val = p.getQuotedString();
+          val[0] = p.getQuotedString();
         } else {
           // read unquoted literal ended by whitespace or endChar (normally '}')
           // there is no escaping.
@@ -127,7 +127,7 @@ public class QueryParsing {
             }
             char c = p.val.charAt(p.pos);
             if (c == endChar || Character.isWhitespace(c)) {
-              val = p.val.substring(valStart, p.pos);
+              val[0] = p.val.substring(valStart, p.pos);
               break;
             }
             p.pos++;
@@ -136,7 +136,7 @@ public class QueryParsing {
 
         if (deref) {  // dereference parameter
           if (params != null) {
-            val = params.get(val);
+            val = params.getParams(val[0]);
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fa55f0c2/solr/core/src/java/org/apache/solr/search/join/BlockJoinChildQParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/join/BlockJoinChildQParser.java b/solr/core/src/java/org/apache/solr/search/join/BlockJoinChildQParser.java
index faf4c5b..0c5c4e8 100644
--- a/solr/core/src/java/org/apache/solr/search/join/BlockJoinChildQParser.java
+++ b/solr/core/src/java/org/apache/solr/search/join/BlockJoinChildQParser.java
@@ -16,10 +16,15 @@
  */
 package org.apache.solr.search.join;
 
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.join.ToChildBlockJoinQuery;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.search.SolrConstantScoreQuery;
+import org.apache.solr.search.SyntaxError;
 
 public class BlockJoinChildQParser extends BlockJoinParentQParser {
 
@@ -36,4 +41,16 @@ public class BlockJoinChildQParser extends BlockJoinParentQParser {
   protected String getParentFilterLocalParamName() {
     return "of";
   }
+  
+  @Override
+  protected Query noClausesQuery() throws SyntaxError {
+    final Query parents = parseParentFilter();
+    final BooleanQuery notParents = new BooleanQuery.Builder()
+        .add(new MatchAllDocsQuery(), Occur.MUST)
+        .add(parents, Occur.MUST_NOT)
+      .build();
+    SolrConstantScoreQuery wrapped = new SolrConstantScoreQuery(getFilter(notParents));
+    wrapped.setCache(false);
+    return wrapped;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fa55f0c2/solr/core/src/java/org/apache/solr/search/join/BlockJoinParentQParser.java
----------------------------------------------------------------------
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 a3b74c0..b91cd62 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
@@ -34,12 +34,11 @@ import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.search.BitsFilteredDocIdSet;
 import org.apache.solr.search.Filter;
 import org.apache.solr.search.QParser;
-import org.apache.solr.search.QueryParsing;
 import org.apache.solr.search.SolrCache;
 import org.apache.solr.search.SolrConstantScoreQuery;
 import org.apache.solr.search.SyntaxError;
 
-public class BlockJoinParentQParser extends QParser {
+public class BlockJoinParentQParser extends FiltersQParser {
   /** implementation detail subject to change */
   public static final String CACHE_NAME="perSegFilter";
 
@@ -47,28 +46,34 @@ public class BlockJoinParentQParser extends QParser {
     return "which";
   }
 
+  @Override
+  protected String getFiltersParamName() {
+    return "filters";
+  }
+
   BlockJoinParentQParser(String qstr, SolrParams localParams, SolrParams params, SolrQueryRequest req) {
     super(qstr, localParams, params, req);
   }
 
-  
-  @Override
-  public Query parse() throws SyntaxError {
+  protected Query parseParentFilter() throws SyntaxError {
     String filter = localParams.get(getParentFilterLocalParamName());
-    String scoreMode = localParams.get("score", ScoreMode.None.name());
     QParser parentParser = subQuery(filter, null);
     Query parentQ = parentParser.getQuery();
+    return parentQ;
+  }
 
-    String queryText = localParams.get(QueryParsing.V);
-    // there is no child query, return parent filter from cache
-    if (queryText == null || queryText.length()==0) {
-                  SolrConstantScoreQuery wrapped = new SolrConstantScoreQuery(getFilter(parentQ));
-                  wrapped.setCache(false);
-                  return wrapped;
-    }
-    QParser childrenParser = subQuery(queryText, null);
-    Query childrenQuery = childrenParser.getQuery();
-    return createQuery(parentQ, childrenQuery, scoreMode);
+  @Override
+  protected Query wrapSubordinateClause(Query subordinate) throws SyntaxError {
+    String scoreMode = localParams.get("score", ScoreMode.None.name());
+    Query parentQ = parseParentFilter();
+    return createQuery(parentQ, subordinate, scoreMode);
+  }
+
+  @Override
+  protected Query noClausesQuery() throws SyntaxError {
+    SolrConstantScoreQuery wrapped = new SolrConstantScoreQuery(getFilter(parseParentFilter()));
+    wrapped.setCache(false);
+    return wrapped;
   }
 
   protected Query createQuery(final Query parentList, Query query, String scoreMode) throws SyntaxError {
@@ -110,7 +115,7 @@ public class BlockJoinParentQParser extends QParser {
       super(childQuery, parentsFilter, scoreMode);
       parentQuery = parentList;
     }
-    
+
     public Query getParentQuery(){
       return parentQuery;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fa55f0c2/solr/core/src/java/org/apache/solr/search/join/FiltersQParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/join/FiltersQParser.java b/solr/core/src/java/org/apache/solr/search/join/FiltersQParser.java
new file mode 100644
index 0000000..6c2172e
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/search/join/FiltersQParser.java
@@ -0,0 +1,184 @@
+/*
+ * 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.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.Query;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.search.QParser;
+import org.apache.solr.search.QueryParsing;
+import org.apache.solr.search.SolrConstantScoreQuery;
+import org.apache.solr.search.SyntaxError;
+
+public class FiltersQParser extends QParser {
+
+  private static final class FiltersQuery extends SolrConstantScoreQuery {
+    
+    private Set<Query> filters;
+
+    private FiltersQuery(SolrQueryRequest req, Set<Query> filters) throws IOException {
+      super(req.getSearcher().getDocSet(new ArrayList<>(filters)).getTopFilter());
+      this.filters = filters;
+    }
+
+    @Override
+    public boolean equals(Object other) {
+      return sameClassAs(other) && filters.equals(((FiltersQuery)other).filters);
+    }
+    
+    @Override
+    public int hashCode() {
+      return 31 * classHash() + filters.hashCode();
+    }
+  }
+
+  protected String getFiltersParamName() {
+    return "param";
+  }
+
+  FiltersQParser(String qstr, SolrParams localParams, SolrParams params, SolrQueryRequest req) {
+    super(qstr, localParams, params, req);
+  }
+
+  @Override
+  public final Query parse() throws SyntaxError {
+    Map<Query,Occur> clauses = clauses();
+    
+    exclude(clauses);
+    
+    int numClauses = 0;
+    BooleanQuery.Builder builder = new BooleanQuery.Builder();
+    numClauses += addQuery(builder, clauses);
+    numClauses += addFilters(builder, clauses);
+    // what about empty query? 
+    return numClauses > 0 ? wrapSubordinateClause(builder.build()) : noClausesQuery();
+  }
+
+  protected Query wrapSubordinateClause(Query subordinate) throws SyntaxError {
+    return subordinate;
+  }
+
+  protected Query noClausesQuery() throws SyntaxError {
+    return new MatchAllDocsQuery();
+  }
+
+  protected int addQuery(BooleanQuery.Builder builder, Map<Query,Occur> clauses) {
+    int cnt=0;
+    for (Map.Entry<Query, Occur> clause: clauses.entrySet()) {
+      if (clause.getValue() == Occur.MUST) {
+        builder.add(clause.getKey(), clause.getValue());
+        cnt++;// shouldn't count more than once 
+      }
+    }
+    return cnt;
+  }
+
+  /** @return number of added clauses */
+  protected int addFilters(BooleanQuery.Builder builder, Map<Query,Occur> clauses) throws SyntaxError {
+    Set<Query> filters = new HashSet<>();
+    for (Map.Entry<Query, Occur> clause: clauses.entrySet()) {
+      if (clause.getValue() == Occur.FILTER) {
+        filters.add(clause.getKey());
+      }
+    }
+    if (!filters.isEmpty()) {
+      try {
+        final SolrConstantScoreQuery intersQuery = new FiltersQuery(req, filters);
+        builder.add( intersQuery, Occur.FILTER);
+      } catch (IOException e) {
+        throw new SyntaxError("Exception occurs while parsing " + stringIncludingLocalParams, e);
+      }
+    }
+    return filters.size();
+  }
+
+  protected void exclude(Map<Query,Occur> clauses) {
+    Set<String> tagsToExclude = new HashSet<>();
+    String excludeTags = localParams.get("excludeTags");
+    if (excludeTags != null) {
+      tagsToExclude.addAll(StrUtils.splitSmart(excludeTags, ','));
+    }
+    @SuppressWarnings("rawtypes")
+    Map tagMap = (Map) req.getContext().get("tags");
+    if (tagMap != null && !tagMap.isEmpty() && !tagsToExclude.isEmpty()) {
+      clauses.keySet().removeAll(excludeSet(tagMap, tagsToExclude));
+    } // else no filters were tagged
+  }
+
+  protected Map<Query,Occur> clauses() throws SyntaxError {
+    String[] params = localParams.getParams(getFiltersParamName());
+    if(params!=null && params.length == 0) { // never happens 
+      throw new SyntaxError("Local parameter "+getFiltersParamName() + 
+                           " is not defined for "+stringIncludingLocalParams);
+    }
+    Map<Query,Occur> clauses = new IdentityHashMap<>();
+    
+    for (String filter : params==null ? new String[0] : params) {
+      if(filter==null || filter.length() == 0) {
+        throw new SyntaxError("Filter '"+filter + 
+                             "' has been picked in "+stringIncludingLocalParams);
+      }
+      // as a side effect, qparser is mapped by tags in req context
+      QParser parser = subQuery(filter, null);
+      Query query = parser.getQuery();
+      clauses.put(query, BooleanClause.Occur.FILTER);
+    }
+    String queryText = localParams.get(QueryParsing.V);
+    if (queryText != null && queryText.length() > 0) {
+      QParser parser = subQuery(queryText, null);
+      clauses.put(parser.getQuery(), BooleanClause.Occur.MUST);
+    }
+    return clauses;
+  }
+
+  private Collection<?> excludeSet(@SuppressWarnings("rawtypes") 
+                                     Map tagMap, Set<String> tagsToExclude) {
+
+    IdentityHashMap<Query,Boolean> excludeSet = new IdentityHashMap<>();
+    for (String excludeTag : tagsToExclude) {
+      Object olst = tagMap.get(excludeTag);
+      // tagMap has entries of List<String,List<QParser>>, but subject to change in the future
+      if (!(olst instanceof Collection)) continue;
+      for (Object o : (Collection<?>)olst) {
+        if (!(o instanceof QParser)) continue;
+        QParser qp = (QParser)o;
+        try {
+          excludeSet.put(qp.getQuery(), Boolean.TRUE);
+        } catch (SyntaxError syntaxError) {
+          // This should not happen since we should only be retrieving a previously parsed query
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, syntaxError);
+        }
+      }
+    }
+    return excludeSet.keySet();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fa55f0c2/solr/core/src/java/org/apache/solr/search/join/FiltersQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/join/FiltersQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/join/FiltersQParserPlugin.java
new file mode 100644
index 0000000..3091ec2
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/search/join/FiltersQParserPlugin.java
@@ -0,0 +1,33 @@
+/*
+ * 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.solr.common.params.SolrParams;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.search.QParser;
+import org.apache.solr.search.QParserPlugin;
+
+public class FiltersQParserPlugin extends QParserPlugin {
+  public static final String NAME = "filters";
+
+  @Override
+  public QParser createParser(String qstr, SolrParams localParams, SolrParams params, SolrQueryRequest req) {
+    QParser parser = new FiltersQParser(qstr, localParams, params, req);
+    return parser;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fa55f0c2/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java b/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
index a52ba56..6112303 100644
--- a/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
+++ b/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
@@ -472,8 +472,63 @@ public class QueryEqualityTest extends SolrTestCaseJ4 {
         "{!parent which=foo_s:parent}dude");
     assertQueryEquals("child", "{!child of=foo_s:parent}dude",
         "{!child of=foo_s:parent}dude");
+    // zero query case 
+    assertQueryEquals(null, "{!parent which=foo_s:parent}",
+        "{!parent which=foo_s:parent}");
+    assertQueryEquals(null, "{!child of=foo_s:parent}",
+        "{!child of=foo_s:parent}");
+    assertQueryEquals(null, "{!parent which='+*:* -foo_s:parent'}",
+        "{!child of=foo_s:parent}");
+    
+    final SolrQueryRequest req = req(
+        "fq","bar_s:baz","fq","{!tag=fqban}bar_s:ban",
+        "ffq","bar_s:baz","ffq","{!tag=ffqban}bar_s:ban");
+    try {
+    assertQueryEquals("filters", req,
+        "{!parent which=foo_s:parent param=$fq}foo_s:bar",
+        "{!parent which=foo_s:parent param=$ffq}foo_s:bar" // differently named params
+        );
+    assertQueryEquals("filters", req,
+        "{!parent which=foo_s:parent param=$fq excludeTags=fqban}foo_s:bar",
+        "{!parent which=foo_s:parent param=$ffq excludeTags=ffqban}foo_s:bar" // differently named params
+        );
+    
+    QueryUtils.checkUnequal(// parent filter is not an equal to child
+        QParser.getParser("{!child of=foo_s:parent}", req).getQuery(),
+        QParser.getParser("{!parent which=foo_s:parent}", req).getQuery());
+    
+    } finally {
+      req.close();
+    }
   }
 
+  public void testFilters() throws Exception {
+    final SolrQueryRequest req = req(
+        "fq","bar_s:baz","fq","{!tag=fqban}bar_s:ban",
+        "ffq","{!tag=ffqbaz}bar_s:baz","ffq","{!tag=ffqban}bar_s:ban");
+    try {
+    assertQueryEquals("filters", req,
+        "{!filters param=$fq}foo_s:bar",
+        "{!filters param=$fq}foo_s:bar",
+        "{!filters param=$ffq}foo_s:bar" // differently named params
+        );
+    assertQueryEquals("filters", req,
+        "{!filters param=$fq excludeTags=fqban}foo_s:bar",
+        "{!filters param=$ffq  excludeTags=ffqban}foo_s:bar" 
+        );
+    assertQueryEquals("filters", req,
+        "{!filters excludeTags=top}{!tag=top v='foo_s:bar'}",
+        "{!filters param=$ffq excludeTags='ffqban,ffqbaz'}" 
+        );
+    QueryUtils.checkUnequal(
+        QParser.getParser("{!filters param=$fq}foo_s:bar", req).getQuery(),
+        QParser.getParser("{!filters param=$fq excludeTags=fqban}foo_s:bar", req).getQuery());    
+    } finally {
+      req.close();
+    }
+  }
+
+
   public void testGraphQuery() throws Exception {
     SolrQueryRequest req = req("from", "node_s",
         "to","edge_s",

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fa55f0c2/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetsWithNestedObjects.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetsWithNestedObjects.java b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetsWithNestedObjects.java
new file mode 100644
index 0000000..5a638ac
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetsWithNestedObjects.java
@@ -0,0 +1,337 @@
+/*
+ * 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.facet;
+
+import org.apache.solr.SolrTestCaseHS;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestJsonFacetsWithNestedObjects extends SolrTestCaseHS{
+
+  @BeforeClass
+  public static void beforeTests() throws Exception {
+    initCore("solrconfig-tlog.xml","schema_latest.xml");
+    indexBooksAndReviews();
+  }
+
+  private static void indexBooksAndReviews() throws Exception {
+    final Client client = Client.localClient();
+    client.deleteByQuery("*:*", null);
+
+    SolrInputDocument book1 = sdoc(
+        "id",         "book1",
+        "type_s",     "book",
+        "title_t",    "The Way of Kings",
+        "author_s",   "Brandon Sanderson",
+        "cat_s",      "fantasy",
+        "pubyear_i",  "2010",
+        "publisher_s","Tor");
+
+    book1.addChildDocument(
+        sdoc(
+            "id", "book1_c1",
+            "type_s", "review",
+            "review_dt", "2015-01-03T14:30:00Z",
+            "stars_i", "5",
+            "author_s", "yonik",
+            "comment_t", "A great start to what looks like an epic series!"));
+
+    book1.addChildDocument(
+        sdoc(
+            "id", "book1_c2",
+            "type_s", "review",
+            "review_dt", "2014-03-15T12:00:00Z",
+            "stars_i", "3",
+            "author_s", "dan",
+            "comment_t", "This book was too long."));
+    client.add(book1, null);
+
+    SolrInputDocument book2 = sdoc(
+        "id",         "book2",
+        "type_s",     "book",
+        "title_t",    "Snow Crash",
+        "author_s",   "Neal Stephenson",
+        "cat_s",      "sci-fi",
+        "pubyear_i",  "1992",
+        "publisher_s","Bantam");
+
+    book2.addChildDocument(
+        sdoc(
+            "id", "book2_c1",
+            "type_s", "review",
+            "review_dt", "2015-01-03T14:30:00Z",
+            "stars_i", "5",
+            "author_s", "yonik",
+            "comment_t", "Ahead of its time... I wonder if it helped inspire The Matrix?"));
+
+    book2.addChildDocument(
+        sdoc(
+            "id", "book2_c2",
+            "type_s", "review",
+            "review_dt", "2015-04-10T9:00:00Z",
+            "stars_i", "2",
+            "author_s", "dan",
+            "comment_t", "A pizza boy for the Mafia franchise? Really?"));
+
+    book2.addChildDocument(
+        sdoc(
+            "id", "book2_c3",
+            "type_s", "review",
+            "review_dt", "2015-06-02T00:00:00Z",
+            "stars_i", "4",
+            "author_s", "mary",
+            "comment_t", "Neal is so creative and detailed! Loved the metaverse!"));
+
+    client.add(book2, null);
+    client.commit();
+  }
+
+  /**
+   * Example from http://yonik.com/solr-nested-objects/
+   * The main query gives us a document list of reviews by author_s:yonik
+   * If we want to facet on the book genre (cat_s field) then we need to
+   * switch the domain from the children (type_s:reviews) to the parents (type_s:books).
+   *
+   * And we get a facet over the books which yonik reviewed
+   *
+   * Note that regardless of which direction we are mapping
+   * (parents to children or children to parents),
+   * we provide a query that defines the complete set of parents in the index.
+   * In these examples, the parent filter is “type_s:book”.
+   */
+  @Test
+  public void testFacetingOnParents() throws Exception {
+    final Client client = Client.localClient();
+    ModifiableSolrParams p = params("rows","10");
+    client.testJQ(params(p, "q", "author_s:yonik", "fl", "id", "fl" , "comment_t"
+        , "json.facet", "{" +
+            "  genres: {" +
+            "    type:terms," +
+            "    field:cat_s," +
+            "    domain: { blockParent : \"type_s:book\" }" +
+            "  }" +
+            "}"
+        )
+        , "response=={numFound:2,start:0,docs:[" +
+            "      {id:book1_c1," +
+            "        comment_t:\"A great start to what looks like an epic series!\"}," +
+            "      {id:book2_c1," +
+            "        comment_t:\"Ahead of its time... I wonder if it helped inspire The Matrix?\"}]}"
+        , "facets=={ count:2," +
+            "genres:{buckets:[ {val:fantasy, count:1}," +
+            "                  {val:sci-fi,  count:1}]}}"
+    );
+  }
+
+  /**
+   * Example from http://yonik.com/solr-nested-objects/
+   * Now lets say we’re displaying the top sci-fi and fantasy books,
+   * and we want to find out who reviews the most books out of our selection.
+   * Since our root implicit facet bucket (formed by the query and filters)
+   * consists of parent documents (books),
+   * we need to switch the facet domain to the children for the author facet.
+   *
+   * Note that regardless of which direction we are mapping
+   * (parents to children or children to parents),
+   * we provide a query that defines the complete set of parents in the index.
+   * In these examples, the parent filter is “type_s:book”.
+   */
+  @Test
+  public void testFacetingOnChildren() throws Exception {
+    final Client client = Client.localClient();
+    ModifiableSolrParams p = params("rows","10");
+    client.testJQ(params(p, "q", "cat_s:(sci-fi OR fantasy)", "fl", "id", "fl" , "title_t"
+        , "json.facet", "{" +
+            "  top_reviewers: {" +
+            "    type:terms," +
+            "    field:author_s," +
+            "    domain: { blockChildren : \"type_s:book\" }" +
+            "  }" +
+            "}"
+        )
+        , "response=={numFound:2,start:0,docs:[" +
+            "      {id:book1," +
+            "        title_t:\"The Way of Kings\"}," +
+            "      {id:book2," +
+            "        title_t:\"Snow Crash\"}]}"
+        , "facets=={ count:2," +
+            "top_reviewers:{buckets:[ {val:dan,     count:2}," +
+            "                         {val:yonik,   count:2}," +
+            "                         {val:mary,    count:1} ]}}"
+    );
+  }
+
+
+  /**
+   * Explicit filter exclusions for rolled up child facets
+   */
+  @Test
+  public void testExplicitFilterExclusions() throws Exception {
+    final Client client = Client.localClient();
+    ModifiableSolrParams p = params("rows","10");
+    client.testJQ(params(p, "q", "{!parent which=type_s:book}comment_t:* %2Bauthor_s:yonik %2Bstars_i:(5 3)"
+        , "fl", "id", "fl" , "title_t"
+        , "json.facet", "{" +
+            "  comments_for_author: {" +
+            "    type:query," +
+            //note: author filter is excluded
+            "    q:\"comment_t:* +stars_i:(5 3)\"," +
+            "    domain: { blockChildren : \"type_s:book\" }," +
+            "    facet:{" +
+            "      authors:{" +
+            "        type:terms," +
+            "        field:author_s," +
+            "        facet: {" +
+            "           in_books: \"unique(_root_)\" }}}}," +
+            "  comments_for_stars: {" +
+            "    type:query," +
+            //note: stars_i filter is excluded
+            "    q:\"comment_t:* +author_s:yonik\"," +
+            "    domain: { blockChildren : \"type_s:book\" }," +
+            "    facet:{" +
+            "      stars:{" +
+            "        type:terms," +
+            "        field:stars_i," +
+            "        facet: {" +
+            "           in_books: \"unique(_root_)\" }}}}}" )
+
+        , "response=={numFound:2,start:0,docs:[" +
+            "      {id:book1," +
+            "        title_t:\"The Way of Kings\"}," +
+            "      {id:book2," +
+            "        title_t:\"Snow Crash\"}]}"
+        , "facets=={ count:2," +
+            "comments_for_author:{" +
+            "  count:3," +
+            "  authors:{" +
+            "    buckets:[ {val:yonik,  count:2, in_books:2}," +
+            "              {val:dan,    count:1, in_books:1} ]}}," +
+            "comments_for_stars:{" +
+            "  count:2," +
+            "  stars:{" +
+            "    buckets:[ {val:5, count:2, in_books:2} ]}}}"
+    );
+  }
+
+  /**
+   * Child level facet exclusions
+   */
+  @Test
+  public void testChildLevelFilterExclusions() throws Exception {
+    final Client client = Client.localClient();
+    ModifiableSolrParams p = params("rows","10");
+    client.testJQ(params(p, "q", "{!parent filters=$child.fq which=type_s:book v=$childquery}"
+        , "childquery", "comment_t:*"
+        , "child.fq", "{!tag=author}author_s:yonik"
+        , "child.fq", "{!tag=stars}stars_i:(5 3)"
+        , "fl", "id", "fl" , "title_t"
+        , "json.facet", "{" +
+            "  comments_for_author: {" +
+            "    type:query," +
+            //note: author filter is excluded
+            "    q:\"{!filters param=$child.fq excludeTags=author v=$childquery}\"," +
+            "    domain: { blockChildren : \"type_s:book\", excludeTags:author }," +
+            "    facet:{" +
+            "      authors:{" +
+            "        type:terms," +
+            "        field:author_s," +
+            "        facet: {" +
+            "           in_books: \"unique(_root_)\" }}}}," +
+            "  comments_for_stars: {" +
+            "    type:query," +
+            //note: stars_i filter is excluded
+            "    q:\"{!filters param=$child.fq excludeTags=stars v=$childquery}\"," +
+            "    domain: { blockChildren : \"type_s:book\" }," +
+            "    facet:{" +
+            "      stars:{" +
+            "        type:terms," +
+            "        field:stars_i," +
+            "        facet: {" +
+            "           in_books: \"unique(_root_)\" }}}}}" )
+
+        , "response=={numFound:2,start:0,docs:[" +
+            "      {id:book1," +
+            "        title_t:\"The Way of Kings\"}," +
+            "      {id:book2," +
+            "        title_t:\"Snow Crash\"}]}"
+        , "facets=={ count:2," +
+            "comments_for_author:{" +
+            "  count:3," +
+            "  authors:{" +
+            "    buckets:[ {val:yonik,  count:2, in_books:2}," +
+            "              {val:dan,    count:1, in_books:1} ]}}," +
+            "comments_for_stars:{" +
+            "  count:2," +
+            "  stars:{" +
+            "    buckets:[ {val:5, count:2, in_books:2} ]}}}"
+    );
+  }
+
+  public void testDomainFilterExclusionsInFilters() throws Exception {
+    final Client client = Client.localClient();
+    ModifiableSolrParams p = params("rows","10");
+    client.testJQ(params(p, "q", "{!parent tag=top filters=$child.fq which=type_s:book v=$childquery}"
+        , "childquery", "comment_t:*"
+        , "child.fq", "{!tag=author}author_s:dan"
+        , "child.fq", "{!tag=stars}stars_i:4"
+        , "fq", "{!tag=top}title_t:Snow\\ Crash"
+        , "fl", "id", "fl" , "title_t"
+        , "json.facet", "{" +
+            "  comments_for_author: {" +
+            "    domain: { excludeTags:\"top\"," + // 1. throwing current parent docset,  
+            "             filter:[\"{!filters param=$child.fq " + // compute children docset from scratch
+            "                       excludeTags=author v=$childquery}\"]"// 3. filter children with exclusions 
+            + "            }," +
+            "    type:terms," +
+            "    field:author_s," +
+            "    facet: {" +
+            "           in_books: \"unique(_root_)\" }"+//}}," +
+            "  }" +
+            //note: stars_i filter is excluded
+            "  ,comments_for_stars: {" +
+            "    domain: { excludeTags:top, " +
+            "          filter:\"{!filters param=$child.fq  excludeTags=stars v=$childquery}\" }," +
+            "    type:terms," +
+            "    field:stars_i," +
+            "    facet: {" +
+            "           in_books: \"unique(_root_)\" }}"+
+            
+            "  ,comments_for_stars_parent_filter: {" +
+            "    domain: { excludeTags:top, " +
+            "          filter:[\"{!filters param=$child.fq  excludeTags=stars v=$childquery}\","
+            + "                \"{!child of=type_s:book filters=$fq}type_s:book\"] }," +
+            "    type:terms," +
+            "    field:stars_i," +
+            "    facet: {" +
+            "           in_books: \"unique(_root_)\" }}"+
+        "}" )
+
+        , "response=={numFound:0,start:0,docs:[]}"
+        , "facets=={ count:0," +
+            "comments_for_author:{" +
+            "    buckets:[ {val:mary,    count:1, in_books:1} ]}," +
+            "comments_for_stars:{" +
+            "    buckets:[ {val:2, count:1, in_books:1}," +
+            "              {val:3, count:1, in_books:1} ]}," +
+            "comments_for_stars_parent_filter:{" +
+            "    buckets:[ {val:2, count:1, in_books:1}" +
+            "               ]}}"
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fa55f0c2/solr/core/src/test/org/apache/solr/search/join/BJQParserTest.java
----------------------------------------------------------------------
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 8c2cec3..6713af7 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
@@ -16,14 +16,7 @@
  */
 package org.apache.solr.search.join;
 
-import org.apache.lucene.search.join.ScoreMode;
-import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.metrics.MetricsMap;
-import org.apache.solr.util.BaseTestHarness;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
+import javax.xml.xpath.XPathConstants;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -33,7 +26,14 @@ import java.util.ListIterator;
 import java.util.Locale;
 import java.util.Map;
 
-import javax.xml.xpath.XPathConstants;
+import org.apache.lucene.search.join.ScoreMode;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.util.BaseTestHarness;
+import org.junit.BeforeClass;
+import org.junit.Test;
 
 public class BJQParserTest extends SolrTestCaseJ4 {
   
@@ -100,7 +100,7 @@ public class BJQParserTest extends SolrTestCaseJ4 {
     List<String[]> block = new ArrayList<>();
     for (String child : klm) {
       block
-          .add(new String[] {"child_s", child, "parentchild_s", parent + child});
+          .add(new String[] {"child_s", child, "parentchild_s", parent + child, "childparent_s", parent});
     }
     Collections.shuffle(block, random());
     addGrandChildren(block);
@@ -150,6 +150,19 @@ public class BJQParserTest extends SolrTestCaseJ4 {
     assertQ(req("q", "{!parent which=\"parent_s:[* TO *]\"}"), sixParents);
   }
   
+  @Test
+  public void testJustParentsFilterInChild() throws IOException {
+    assertQ(req("q", "{!child of=\"parent_s:[* TO *]\"}",
+          "fq", "childparent_s:"+abcdef[random().nextInt(abcdef.length)],
+        "indent","on"), 
+        "//*[@numFound='"+klm.length+"']", //for any parent we have all three children
+        "//doc/arr[@name='child_s']/str='"+klm[0]+"'",
+        "//doc/arr[@name='child_s']/str='"+klm[1]+"'",
+        "//doc/arr[@name='child_s']/str='"+klm[2]+"'"
+        );
+    assert klm.length==3 : "change asserts pls "+klm;
+  }
+  
   private final static String beParents[] = new String[] {"//*[@numFound='2']",
       "//doc/arr[@name=\"parent_s\"]/str='b'",
       "//doc/arr[@name=\"parent_s\"]/str='e'"};
@@ -326,6 +339,111 @@ public class BJQParserTest extends SolrTestCaseJ4 {
   public void nullInit() {
     new BlockJoinParentQParserPlugin().init(null);
   }
-  
+
+  private final static String eParent[] = new String[]{"//*[@numFound='1']",
+      "//doc/arr[@name=\"parent_s\"]/str='e'"};
+
+  @Test
+  public void testToParentFilters() {
+    assertQ(
+        req("fq", "{!parent filters=$child.fq which=$pq v=$chq}\"",
+            "q", "parent_s:(e b)",
+            "child.fq", "+childparent_s:e +child_s:l",
+            "chq", "child_s:[* TO *]",
+            "pq", "parent_s:[* TO *]"), eParent);
+
+    assertQ(
+        req("fq", "{!parent filters=$child.fq which=$pq v=$chq}\"",
+            "q", "parent_s:(e b)",
+            "child.fq", "childparent_s:e",
+            "child.fq", "child_s:l",
+            "chq", "child_s:[* TO *]",
+            "pq", "parent_s:[* TO *]"), eParent);
+  }
+
+  @Test
+  public void testToChildFilters() {
+    assertQ(
+        req("fq", "{!child of=$pq filters=$parent.fq  v=$pq}\"",
+            "q", "child_s:(l m)",
+            "parent.fq", "+parent_s:(d c)",
+            "pq", "parent_s:[* TO *]"),
+        "//*[@numFound='4']",
+        "//doc/arr[@name=\"parentchild_s\"]/str='dl'",
+        "//doc/arr[@name=\"parentchild_s\"]/str='dm'",
+        "//doc/arr[@name=\"parentchild_s\"]/str='cl'",
+        "//doc/arr[@name=\"parentchild_s\"]/str='cm'"
+    );
+
+    assertQ(
+        req("fq", "{!child of=$pq filters=$parent.fq  v=$pq}\"",
+            "q", "child_s:(l m)",
+            "parent.fq", "+parent_s:(d c)",
+            "parent.fq", "+parent_s:(c a)",
+            "pq", "parent_s:[* TO *]"),
+        "//*[@numFound='2']",
+        "//doc/arr[@name=\"parentchild_s\"]/str='cl'",
+        "//doc/arr[@name=\"parentchild_s\"]/str='cm'"
+    );
+  }
+
+  private final static String elChild[] = new String[]{"//*[@numFound='1']",
+      "//doc[" +
+          "arr[@name=\"child_s\"]/str='l' and child::arr[@name=\"childparent_s\"]/str='e']"};
+
+
+  @Test
+  public void testFilters() {
+    assertQ(
+        req("q", "{!filters param=$child.fq v=$gchq}",
+            "child.fq", "childparent_s:e",
+            "child.fq", "child_s:l",
+            "gchq", "child_s:[* TO *]"), elChild);
+
+    assertQ(
+        req("q", "{!filters param=$child.fq excludeTags=firstTag v=$gchq}",
+            "child.fq", "{!tag=zeroTag,firstTag}childparent_s:e",
+            "child.fq", "{!tag=secondTag}child_s:l",
+            "gchq", "child_s:[* TO *]"), "//*[@numFound='6']");
+
+    assertQ(
+        req("q", "{!filters param=$child.fq excludeTags=secondTag v=$gchq}",
+            "child.fq", "{!tag=firstTag}childparent_s:e",
+            "child.fq", "{!tag=secondTag}child_s:l",
+            "gchq", "child_s:[* TO *]"), "//*[@numFound='3']");
+
+    assertQ(req("q",
+             random().nextBoolean() ? "{!filters param=$child.fq excludeTags=firstTag,secondTag v=$gchq}" :
+               random().nextBoolean() ? "{!filters param=$thereAreNoLikeThese v=$gchq}" :
+                 "{!filters v=$gchq}" ,
+            "child.fq", "{!tag=firstTag}childparent_s:e",
+            "child.fq", "{!tag=secondTag}child_s:l",
+            "gchq", "child_s:[* TO *]"), "//*[@numFound='18']");
+    
+    assertQEx("expecting exception on weird param",
+        req("q", "{!filters v=$gchq param=}\"" ,
+            "gchq", "child_s:[* TO *]"
+       ),ErrorCode.BAD_REQUEST);
+    
+    assertQ( // omit main query
+        req("q", "{!filters param=$child.fq}",
+            "child.fq", "{!tag=firstTag}childparent_s:(e f)",
+            "child.fq", "{!tag=secondTag}child_s:l"), "//*[@numFound='2']");
+    
+    assertQ( // all excluded, matching all 
+        req("q", "{!filters param=$child.fq excludeTags=firstTag,secondTag}",
+            "child.fq", "{!tag=firstTag}childparent_s:(e f)",
+            "child.fq", "{!tag=secondTag}child_s:l"), "//*[@numFound='42']");
+    
+    assertQ(req("q", // excluding top level
+            "{!filters param=$child.fq excludeTags=bot,top v=$gchq}" ,
+       "child.fq", "{!tag=secondTag}child_s:l", // 6 ls remains
+       "gchq", "{!tag=top}childparent_s:e"), "//*[@numFound='6']");
+    
+    assertQ(req("q", // top and filter are excluded, got zero result, but is it right? 
+        "{!filters excludeTags=bot,secondTag,top v=$gchq}" ,
+         "child.fq", "{!tag=secondTag}child_s:l",
+         "gchq", "{!tag=top}childparent_s:e"), "//*[@numFound='42']");
+  }
 }