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 2017/04/28 11:20:02 UTC

lucene-solr:branch_6x: SOLR-10521: adding sort=childfield(field, $q) asc for {!parent} query.

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_6x 3866c72fe -> 189e50d9b


SOLR-10521: adding sort=childfield(field,$q) asc for {!parent} query.


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

Branch: refs/heads/branch_6x
Commit: 189e50d9bcba4bd39e31fe6007ac56c1a882a61b
Parents: 3866c72
Author: Mikhail Khludnev <mk...@apache.org>
Authored: Sat Apr 22 14:39:33 2017 +0300
Committer: Mikhail Khludnev <mk...@apache.org>
Committed: Fri Apr 28 14:00:18 2017 +0300

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   1 +
 .../apache/solr/search/ValueSourceParser.java   |   4 +-
 .../search/join/BlockJoinParentQParser.java     |  11 +-
 .../join/ChildFieldValueSourceParser.java       | 197 +++++++++++++++++++
 .../apache/solr/search/QueryEqualityTest.java   |   9 +
 .../search/join/TestCloudNestedDocsSort.java    | 183 +++++++++++++++++
 .../solr/search/join/TestNestedDocsSort.java    | 141 +++++++++++++
 7 files changed, 542 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/189e50d9/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index d5bee36..c96b821 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -86,6 +86,7 @@ New Features
 
 * SOLR-10507: Core Admin status command to emit collection details of each core (noble)
 
+* SOLR-10521: introducing sort=childfield(field) asc for searching by {!parent} (Mikhail Khludnev) 
 
 Optimizations
 ----------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/189e50d9/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java b/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
index fd65fac..d1b7a51 100644
--- a/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
+++ b/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
@@ -74,6 +74,7 @@ import org.apache.solr.search.function.distance.HaversineFunction;
 import org.apache.solr.search.function.distance.SquaredEuclideanFunction;
 import org.apache.solr.search.function.distance.StringDistanceFunction;
 import org.apache.solr.search.function.distance.VectorDistanceFunction;
+import org.apache.solr.search.join.ChildFieldValueSourceParser;
 import org.apache.solr.util.DateMathParser;
 import org.apache.solr.util.plugin.NamedListInitializedPlugin;
 import org.locationtech.spatial4j.distance.DistanceUtils;
@@ -975,7 +976,8 @@ public abstract class ValueSourceParser implements NamedListInitializedPlugin {
     });
 
     addParser("agg_percentile", new PercentileAgg.Parser());
-
+    
+    addParser("childfield", new ChildFieldValueSourceParser());
   }
 
   ///////////////////////////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/189e50d9/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 8f36dd2..a3b74c0 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
@@ -41,7 +41,7 @@ import org.apache.solr.search.SyntaxError;
 
 public class BlockJoinParentQParser extends QParser {
   /** implementation detail subject to change */
-  public String CACHE_NAME="perSegFilter";
+  public static final String CACHE_NAME="perSegFilter";
 
   protected String getParentFilterLocalParamName() {
     return "which";
@@ -51,6 +51,7 @@ public class BlockJoinParentQParser extends QParser {
     super(qstr, localParams, params, req);
   }
 
+  
   @Override
   public Query parse() throws SyntaxError {
     String filter = localParams.get(getParentFilterLocalParamName());
@@ -75,7 +76,11 @@ public class BlockJoinParentQParser extends QParser {
   }
 
   BitDocIdSetFilterWrapper getFilter(Query parentList) {
-    SolrCache parentCache = req.getSearcher().getCache(CACHE_NAME);
+    return getCachedFilter(req, parentList);
+  }
+
+  static BitDocIdSetFilterWrapper getCachedFilter(final SolrQueryRequest request, Query parentList) {
+    SolrCache parentCache = request.getSearcher().getCache(CACHE_NAME);
     // lazily retrieve from solr cache
     Filter filter = null;
     if (parentCache != null) {
@@ -93,7 +98,7 @@ public class BlockJoinParentQParser extends QParser {
     return result;
   }
 
-  private BitSetProducer createParentFilter(Query parentQ) {
+  private static BitSetProducer createParentFilter(Query parentQ) {
     return new QueryBitSetProducer(parentQ);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/189e50d9/solr/core/src/java/org/apache/solr/search/join/ChildFieldValueSourceParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/join/ChildFieldValueSourceParser.java b/solr/core/src/java/org/apache/solr/search/join/ChildFieldValueSourceParser.java
new file mode 100644
index 0000000..a31d36f
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/search/join/ChildFieldValueSourceParser.java
@@ -0,0 +1,197 @@
+/*
+ * 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.lang.invoke.MethodHandles;
+import java.util.Map;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.FieldComparator;
+import org.apache.lucene.search.LeafFieldComparator;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.SortField.Type;
+import org.apache.lucene.search.join.BitSetProducer;
+import org.apache.lucene.search.join.QueryBitSetProducer;
+import org.apache.lucene.search.join.ToParentBlockJoinSortField;
+import org.apache.lucene.util.BytesRef;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.search.FunctionQParser;
+import org.apache.solr.search.SyntaxError;
+import org.apache.solr.search.ValueSourceParser;
+import org.apache.solr.search.join.BlockJoinParentQParser.AllParentsAware;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ChildFieldValueSourceParser extends ValueSourceParser {
+  
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final class BlockJoinSortFieldValueSource extends ValueSource {
+    private static final class BytesToStringComparator extends FieldComparator<String> {
+      private final FieldComparator<BytesRef> byteRefs;
+
+      private BytesToStringComparator(FieldComparator<BytesRef> byteRefs) {
+        this.byteRefs = byteRefs;
+      }
+
+      @Override
+      public int compare(int slot1, int slot2) {
+        return byteRefs.compare(slot1, slot2);
+      }
+
+      @Override
+      public void setTopValue(String value) {
+        byteRefs.setTopValue(new BytesRef(value));
+      }
+
+      @Override
+      public String value(int slot) {
+        return byteRefs.value(slot).utf8ToString();
+      }
+
+      @Override
+      public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
+        return byteRefs.getLeafComparator(context);
+      }
+    }
+
+    private final BitSetProducer childFilter;
+    private final BitSetProducer parentFilter;
+    private final SchemaField childField;
+
+    private BlockJoinSortFieldValueSource(BitSetProducer childFilter, BitSetProducer parentFilter,
+        SchemaField childField) {
+      this.childFilter = childFilter;
+      this.parentFilter = parentFilter;
+      this.childField = childField;
+    }
+
+
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + ((childField == null) ? 0 : childField.hashCode());
+      result = prime * result + ((childFilter == null) ? 0 : childFilter.hashCode());
+      result = prime * result + ((parentFilter == null) ? 0 : parentFilter.hashCode());
+      return result;
+    }
+
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) return true;
+      if (obj == null) return false;
+      if (getClass() != obj.getClass()) return false;
+      BlockJoinSortFieldValueSource other = (BlockJoinSortFieldValueSource) obj;
+      if (childField == null) {
+        if (other.childField != null) return false;
+      } else if (!childField.equals(other.childField)) return false;
+      if (childFilter == null) {
+        if (other.childFilter != null) return false;
+      } else if (!childFilter.equals(other.childFilter)) return false;
+      if (parentFilter == null) {
+        if (other.parentFilter != null) return false;
+      } else if (!parentFilter.equals(other.parentFilter)) return false;
+      return true;
+    }
+    
+
+    @Override
+    public String toString() {
+      return "BlockJoinSortFieldValueSource [childFilter=" + childFilter + ", parentFilter=" + parentFilter
+          + ", childField=" + childField + "]";
+    }
+
+    @Override
+    public SortField getSortField(boolean reverse) {
+      final Type type = childField.getSortField(reverse).getType();
+        return new ToParentBlockJoinSortField(childField.getName(), 
+            type, reverse, 
+            parentFilter, childFilter) {
+          @Override
+          public FieldComparator<?> getComparator(int numHits, int sortPos) {
+            final FieldComparator<?> comparator = super.getComparator(numHits, sortPos);
+            return type ==Type.STRING ?  new BytesToStringComparator((FieldComparator<BytesRef>) comparator): comparator;
+          }
+        };
+    }
+
+    @Override
+    public String description() {
+      return NAME + " for " + childField.getName() +" of "+ query(childFilter);
+    }
+
+    private String query(BitSetProducer bits) {
+      return (bits instanceof QueryBitSetProducer) ? ((QueryBitSetProducer) bits).getQuery().toString()
+          : bits.toString();
+    }
+
+    @Override
+    public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
+      throw new UnsupportedOperationException(this + " is only for sorting");
+    }
+  }
+
+  public static final String NAME = "childfield";
+
+  @Override
+  public ValueSource parse(FunctionQParser fp) throws SyntaxError {
+    
+    final String sortFieldName = fp.parseArg();
+    final Query query;
+    if (fp.hasMoreArguments()){
+      query = fp.parseNestedQuery();
+    }else{
+      query = fp.subQuery(fp.getParam(CommonParams.Q), BlockJoinParentQParserPlugin.NAME).getQuery();
+    }
+    
+    BitSetProducer parentFilter;
+    BitSetProducer childFilter;
+    SchemaField sf;
+    try {
+      AllParentsAware bjQ;
+      if (!(query instanceof AllParentsAware)) {
+        throw new SyntaxError("expect a reference to block join query "+
+              AllParentsAware.class.getSimpleName()+" in "+fp.getString());
+      }
+      bjQ = (AllParentsAware) query;
+      
+      parentFilter = BlockJoinParentQParser.getCachedFilter(fp.getReq(), bjQ.getParentQuery()).filter;
+      childFilter = BlockJoinParentQParser.getCachedFilter(fp.getReq(), bjQ.getChildQuery()).filter;
+
+      if (sortFieldName==null || sortFieldName.equals("")) {
+        throw new SyntaxError ("field is omitted in "+fp.getString());
+      }
+      
+      sf = fp.getReq().getSchema().getFieldOrNull(sortFieldName);
+      if (null == sf) {
+        throw new SyntaxError
+          (NAME+" sort param field \""+ sortFieldName+"\" can't be found in schema");
+      }
+    } catch (SyntaxError e) {
+      log.error("can't parse "+fp.getString(), e);
+      throw e;
+    }
+    return new BlockJoinSortFieldValueSource(childFilter, parentFilter, sf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/189e50d9/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 22d7586..c44cc1e 100644
--- a/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
+++ b/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
@@ -1122,4 +1122,13 @@ public class QueryEqualityTest extends SolrTestCaseJ4 {
     assertFalse(equals);
   }
 
+  public void testChildField() throws Exception {
+    final SolrQueryRequest req = req("q", "{!parent which=type_s1:parent}whatever_s1:foo");
+    try {
+      assertFuncEquals(req,
+          "childfield(name_s1,$q)", "childfield(name_s1,$q)");
+    } finally {
+      req.close();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/189e50d9/solr/core/src/test/org/apache/solr/search/join/TestCloudNestedDocsSort.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/join/TestCloudNestedDocsSort.java b/solr/core/src/test/org/apache/solr/search/join/TestCloudNestedDocsSort.java
new file mode 100644
index 0000000..22aa052
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/search/join/TestCloudNestedDocsSort.java
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.search.join;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.cloud.AbstractDistribZkTestBase;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestCloudNestedDocsSort extends SolrCloudTestCase {
+
+  private static ArrayList<String> vals = new ArrayList<>();
+  private static CloudSolrClient client;
+  private static int maxDocs;
+  private static String matchingParent;
+  private static String matchingChild;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    for (int i=0; i<10+random().nextInt(20); i++) {
+      vals.add(""+Integer.toString(random().nextInt(1000000), Character.MAX_RADIX));
+    }
+    
+    final Path configDir = Paths.get(TEST_HOME(), "collection1", "conf");
+
+    String configName = "solrCloudCollectionConfig";
+    int nodeCount = 5;
+    configureCluster(nodeCount)
+       .addConfig(configName, configDir)
+       .configure();
+    
+    int shards = 2;
+    int replicas = 2 ;
+    CollectionAdminRequest.createCollection("collection1", configName, shards, replicas)
+        .withProperty("config", "solrconfig-minimal.xml")
+        .withProperty("schema", "schema.xml")
+        .process(cluster.getSolrClient());
+
+    client = cluster.getSolrClient();
+    client.setDefaultCollection("collection1");
+    
+    ZkStateReader zkStateReader = client.getZkStateReader();
+    AbstractDistribZkTestBase.waitForRecoveriesToFinish("collection1", zkStateReader, true, true, 30);
+    
+    {
+      List<SolrInputDocument> docs = new ArrayList<>();
+      int parentsNum = 10+random().nextInt(20);
+      for (int i=0; i<parentsNum || (matchingParent==null ||matchingChild==null); i++) {
+        final String parentTieVal = "" + random().nextInt(5);
+        final String parentId = ""+random().nextInt();
+        final SolrInputDocument parent = new SolrInputDocument("id", parentId,
+            "type_s", "parent",
+            "parentTie_s1", parentTieVal,
+            "parent_id_s1", parentId
+            );
+        final List<String> parentFilter = addValsField(parent, "parentFilter_s");
+        final int kids = usually() ? atLeast(20) : 0;
+        for(int c = 0; c< kids; c++){
+          SolrInputDocument child = new SolrInputDocument("id", ""+random().nextInt(),
+              "type_s", "child",
+              "parentTie_s1", parentTieVal,
+              "val_s1", Integer.toString(random().nextInt(1000), Character.MAX_RADIX)+"" ,
+              "parent_id_s1", parentId);
+          child.addField("parentFilter_s", parentFilter);
+          final List<String> chVals = addValsField(child, "childFilter_s");
+          parent.addChildDocument(child );
+
+          // let's pickup at least matching child
+          final boolean canPickMatchingChild = !chVals.isEmpty() && !parentFilter.isEmpty();
+          final boolean haveNtPickedMatchingChild = matchingParent==null ||matchingChild==null;
+          if (canPickMatchingChild && haveNtPickedMatchingChild && usually()) {
+            matchingParent = (String) parentFilter.iterator().next();
+            matchingChild = (String) chVals.iterator().next();
+          }
+        }
+        maxDocs+=parent.getChildDocumentCount()+1;
+        docs.add(parent);
+      }
+      client.add(docs);
+      client.commit();
+    }
+  }
+
+  @Test
+  public void test() throws SolrServerException, IOException {
+    final boolean asc = random().nextBoolean();
+    final String dir = asc ? "asc": "desc";
+    final String parentFilter = "+parentFilter_s:("+matchingParent+" "+anyValsSpaceDelim(2)+")^=0";
+    String childFilter = "+childFilter_s:("+matchingChild+" "+anyValsSpaceDelim(4)+")^=0";
+    final String fl = "id,type_s,parent_id_s1,val_s1,score,parentFilter_s,childFilter_s,parentTie_s1";
+    String sortClause = "val_s1 "+dir+", "+"parent_id_s1 "+ascDesc();
+    if(rarely()) {
+      sortClause ="parentTie_s1 "+ascDesc()+","+sortClause;
+    }
+    final SolrQuery q = new SolrQuery("q", "+type_s:child^=0 "+parentFilter+" "+
+          childFilter ,
+        "sort", sortClause, 
+        "rows", ""+maxDocs,
+        "fl",fl);
+
+    final QueryResponse children = client.query(q);
+    
+    final SolrQuery bjq = new SolrQuery("q", "{!parent which=type_s:parent}(+type_s:child^=0 "+parentFilter+" "+
+        childFilter+")",
+       "sort", sortClause.replace("val_s1 ", "childfield(val_s1)"),
+        "rows", ""+maxDocs, "fl", fl);
+
+    final QueryResponse parents = client.query(bjq);
+    
+    Set<String> parentIds = new LinkedHashSet<>();
+    assertTrue("it can never be empty for sure", parents.getResults().size()>0);
+    for(Iterator<SolrDocument> parentIter = parents.getResults().iterator(); parentIter.hasNext();) {
+      for (SolrDocument child : children.getResults()) {
+        assertEquals("child", child.getFirstValue("type_s"));
+        final String parentId = (String) child.getFirstValue("parent_id_s1");
+        if( parentIds.add(parentId) ) { // in children the next parent appears, it should be next at parents 
+          final SolrDocument parent = parentIter.next();
+          assertEquals("parent", parent.getFirstValue("type_s"));
+          final String actParentId = ""+ parent.get("id");
+          if (!actParentId.equals(parentId)) {
+            final String chDump = children.toString().replace("SolrDocument","\nSolrDocument");
+            System.out.println("\n\n"+chDump.substring(0,5000)+"\n\n");
+            System.out.println("\n\n"+chDump
+                +"\n\n");
+          }
+          assertEquals(actParentId, parentId);
+        }
+      }
+    }
+
+    
+  }
+
+  private String ascDesc() {
+    return random().nextBoolean() ? "asc": "desc";
+  }
+
+  protected String anyValsSpaceDelim(int howMany) {
+    Collections.shuffle(vals, random());
+    return vals.subList(0, howMany).toString().replaceAll("[,\\[\\]]", "");
+  }
+
+  protected static List<String> addValsField(final SolrInputDocument parent, final String field) {
+    Collections.shuffle(vals, random());
+    final ArrayList<String> values = new ArrayList<>(vals.subList(0, 1+random().nextInt(vals.size()-1)));
+    assertFalse(values.isEmpty());
+    parent.addField(field, values);
+    return values;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/189e50d9/solr/core/src/test/org/apache/solr/search/join/TestNestedDocsSort.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/join/TestNestedDocsSort.java b/solr/core/src/test/org/apache/solr/search/join/TestNestedDocsSort.java
new file mode 100644
index 0000000..3e78965
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/search/join/TestNestedDocsSort.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.search.join;
+
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.search.SolrCache;
+import org.apache.solr.search.SortSpec;
+import org.apache.solr.search.SortSpecParsing;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestNestedDocsSort extends SolrTestCaseJ4 {
+    
+    @BeforeClass
+    public static void beforeClass() throws Exception {
+      initCore("solrconfig.xml", "schema.xml");
+    }
+    
+    public void testEquality(){
+      parseAssertEq("childfield(name_s1,$q) asc", "childfield(name_s1,$q) asc");
+      parseAssertEq("childfield(name_s1,$q) asc", "childfield(name_s1) asc");  
+      parseAssertEq("childfield(name_s1,$q) asc", "childfield(name_s1,) asc");   
+      
+      parseAssertNe("childfield(name_s1,$q) asc", "childfield(name_s1,$q) desc");
+      parseAssertNe("childfield(name_s1,$q) asc", "childfield(surname_s1,$q) asc");
+      parseAssertNe("childfield(name_s1,$q) asc", "childfield(surname_s1,$q2) desc");
+    }
+    
+    @AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/LUCENE-7798")
+    public void testEqualityUpToBlockJoin(){
+      parseAssertNe("childfield(name_s1,$q) asc","childfield(name_s1,$q2) asc");
+    }
+    
+    @Test(expected=SolrException.class)
+    public void testNotBjqReference(){
+      parse("childfield(name_s1,$notbjq) asc");
+    }
+    
+    // root cause is swallowed, but it's logged there. 
+    @Test(expected=SolrException.class)
+    public void testOmitFieldWithComma(){
+      parse("childfield(,$q)  asc");
+    }
+    @Test(expected=SolrException.class)
+    public void testOmitField(){
+      parse("childfield($q)  asc");
+    }
+
+    @Test(expected=SolrException.class)
+    public void testForgetEverything(){
+      parse("childfield() asc");
+    }
+    
+    @Test(expected=SolrException.class)
+    public void testEvenBraces(){
+      parse("childfield asc");
+    }
+    
+    @Test(expected=SolrException.class)
+    public void testAbsentField(){
+      parse("childfield(NEVER_SEEN_IT,$q) asc");
+    }
+    
+    @Test(expected=SolrException.class)
+    public void testOmitOrder(){
+      parse("childfield(name_s1,$q)");
+    }
+    
+    private void parseAssertEq(String sortField, String sortField2) {
+      assertEq(parse(sortField), parse(sortField2));
+    }
+    
+    private void assertEq(SortField sortField, SortField sortField2) {
+      assertEquals(sortField, sortField2);
+      assertEquals(sortField.hashCode(), sortField2.hashCode());
+    }
+    
+    private void parseAssertNe(String sortField, String sortField2) {
+      assertFalse(parse(sortField).equals(parse(sortField2)));
+    }
+
+    private SortField parse(String a) {
+        final SolrQueryRequest req = req("q", "{!parent which=type_s1:parent}whatever_s1:foo",
+            "q2", "{!parent which=type_s1:parent}nomater_s1:what",
+            "notbjq", "foo_s1:bar");
+        try {
+        final SortSpec spec = SortSpecParsing.parseSortSpec(a,
+            req);
+        assertNull(spec.getSchemaFields().get(0));
+        final Sort sort = spec.getSort();
+        final SortField field = sort.getSort()[0];
+        assertNotNull(field);
+        return field;
+      } finally {
+        req.close();
+      }
+    }
+
+    public void testCachehits(){
+      final SolrQueryRequest req = req();
+      try {
+        @SuppressWarnings("rawtypes")
+        final SolrCache cache = req.getSearcher().getCache("perSegFilter");
+        assertNotNull(cache);
+        @SuppressWarnings("unchecked")
+        final NamedList<Number> state = (NamedList<Number>)cache.getStatistics();
+        System.out.println(state);
+        Number before = (Number) state.get("lookups");
+        parse("childfield(name_s1,$q) asc");
+        Number after = (Number) cache.getStatistics().get("lookups");
+        assertEquals("parsing bjq lookups parent filter,"
+            + "parsing sort spec lookups parent and child filters, "
+            + "hopefully for the purpose",3, after.intValue()-before.intValue());
+      } finally {
+        req.close();
+      }
+    }
+    
+    
+
+    
+}