You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ds...@apache.org on 2018/09/05 14:50:20 UTC

lucene-solr:master: SOLR-12722: [child] transformer now processes 'fl'

Repository: lucene-solr
Updated Branches:
  refs/heads/master c8b47e202 -> e4f256be1


SOLR-12722: [child] transformer now processes 'fl'


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

Branch: refs/heads/master
Commit: e4f256be15ca44f12a4aecb32c13d1ab2617cc00
Parents: c8b47e2
Author: David Smiley <ds...@apache.org>
Authored: Wed Sep 5 10:50:14 2018 -0400
Committer: David Smiley <ds...@apache.org>
Committed: Wed Sep 5 10:50:14 2018 -0400

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  4 ++
 .../response/transform/ChildDocTransformer.java | 18 +++++--
 .../transform/ChildDocTransformerFactory.java   | 30 +++++++++++-
 .../solr/response/transform/DocTransformer.java | 23 +++++++++
 .../transform/RawValueTransformerFactory.java   | 20 +-------
 .../transform/TestChildDocTransformer.java      | 51 +++++++++++++++++---
 .../src/transforming-result-documents.adoc      |  1 +
 .../solr/client/solrj/SolrExampleTests.java     |  4 +-
 8 files changed, 116 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e4f256be/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index c0c6dbd..6bb1299 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -204,6 +204,10 @@ New Features
 * SOLR-12716: NodeLostTrigger should support deleting replicas from lost nodes by setting preferredOperation=deletenode.
   (shalin)
 
+* SOLR-12722: The [child] transformer now takes an 'fl' param to specify which fields to return.  It will evaluate
+  doc transformers if present.  In 7.5 a missing 'fl' defaults to the current behavior of all fields, but in 8.0
+  defaults to the top/request "fl". (Moshe Bla, David Smiley)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e4f256be/solr/core/src/java/org/apache/solr/response/transform/ChildDocTransformer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/response/transform/ChildDocTransformer.java b/solr/core/src/java/org/apache/solr/response/transform/ChildDocTransformer.java
index bffbaf2..2628f75 100644
--- a/solr/core/src/java/org/apache/solr/response/transform/ChildDocTransformer.java
+++ b/solr/core/src/java/org/apache/solr/response/transform/ChildDocTransformer.java
@@ -54,17 +54,16 @@ class ChildDocTransformer extends DocTransformer {
   private final DocSet childDocSet;
   private final int limit;
   private final boolean isNestedSchema;
+  private final SolrReturnFields childReturnFields;
 
-  //TODO ought to be provided/configurable
-  private final SolrReturnFields childReturnFields = new SolrReturnFields();
-
-  ChildDocTransformer(String name, BitSetProducer parentsFilter,
-                      DocSet childDocSet, boolean isNestedSchema, int limit) {
+  ChildDocTransformer(String name, BitSetProducer parentsFilter, DocSet childDocSet,
+                      SolrReturnFields returnFields, boolean isNestedSchema, int limit) {
     this.name = name;
     this.parentsFilter = parentsFilter;
     this.childDocSet = childDocSet;
     this.limit = limit;
     this.isNestedSchema = isNestedSchema;
+    this.childReturnFields = returnFields!=null? returnFields: new SolrReturnFields();
   }
 
   @Override
@@ -73,6 +72,9 @@ class ChildDocTransformer extends DocTransformer {
   }
 
   @Override
+  public boolean needsSolrIndexSearcher() { return true; }
+
+  @Override
   public void transform(SolrDocument rootDoc, int rootDocId) {
     // note: this algorithm works if both if we have have _nest_path_  and also if we don't!
 
@@ -132,6 +134,12 @@ class ChildDocTransformer extends DocTransformer {
 
           // load the doc
           SolrDocument doc = searcher.getDocFetcher().solrDoc(docId, childReturnFields);
+          if(childReturnFields.getTransformer() != null) {
+            if(childReturnFields.getTransformer().context == null) {
+              childReturnFields.getTransformer().setContext(context);
+            }
+            childReturnFields.getTransformer().transform(doc, docId);
+          }
 
           if (isAncestor) {
             // if this path has pending child docs, add them.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e4f256be/solr/core/src/java/org/apache/solr/response/transform/ChildDocTransformerFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/response/transform/ChildDocTransformerFactory.java b/solr/core/src/java/org/apache/solr/response/transform/ChildDocTransformerFactory.java
index 2478c48..82be49d 100644
--- a/solr/core/src/java/org/apache/solr/response/transform/ChildDocTransformerFactory.java
+++ b/solr/core/src/java/org/apache/solr/response/transform/ChildDocTransformerFactory.java
@@ -33,6 +33,7 @@ import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.search.DocSet;
 import org.apache.solr.search.QParser;
+import org.apache.solr.search.SolrReturnFields;
 import org.apache.solr.search.SyntaxError;
 
 import static org.apache.solr.schema.IndexSchema.NEST_PATH_FIELD_NAME;
@@ -57,12 +58,28 @@ public class ChildDocTransformerFactory extends TransformerFactory {
 
   static final char PATH_SEP_CHAR = '/';
   static final char NUM_SEP_CHAR = '#';
+  private static final ThreadLocal<Boolean> recursionCheckThreadLocal = ThreadLocal.withInitial(() -> Boolean.FALSE);
   private static final BooleanQuery rootFilter = new BooleanQuery.Builder()
       .add(new BooleanClause(new MatchAllDocsQuery(), BooleanClause.Occur.MUST))
       .add(new BooleanClause(new DocValuesFieldExistsQuery(NEST_PATH_FIELD_NAME), BooleanClause.Occur.MUST_NOT)).build();
 
   @Override
   public DocTransformer create(String field, SolrParams params, SolrQueryRequest req) {
+    if(recursionCheckThreadLocal.get()) {
+      // this is a recursive call by SolrReturnFields, see ChildDocTransformerFactory#createChildDocTransformer
+      return new DocTransformer.NoopFieldTransformer();
+    } else {
+      try {
+        // transformer is yet to be initialized in this thread, create it
+        recursionCheckThreadLocal.set(true);
+        return createChildDocTransformer(field, params, req);
+      } finally {
+        recursionCheckThreadLocal.set(false);
+      }
+    }
+  }
+
+  private DocTransformer createChildDocTransformer(String field, SolrParams params, SolrQueryRequest req) {
     SchemaField uniqueKeyField = req.getSchema().getUniqueKeyField();
     if (uniqueKeyField == null) {
       throw new SolrException( ErrorCode.BAD_REQUEST,
@@ -106,9 +123,20 @@ public class ChildDocTransformerFactory extends TransformerFactory {
       }
     }
 
+    String childReturnFields = params.get("fl");
+    SolrReturnFields childSolrReturnFields;
+    if(childReturnFields != null) {
+      childSolrReturnFields = new SolrReturnFields(childReturnFields, req);
+    } else if(req.getSchema().getDefaultLuceneMatchVersion().major < 8) {
+      // ensure backwards for versions prior to SOLR 8
+      childSolrReturnFields = new SolrReturnFields();
+    } else {
+      childSolrReturnFields = new SolrReturnFields(req);
+    }
+
     int limit = params.getInt( "limit", 10 );
 
-    return new ChildDocTransformer(field, parentsFilter, childDocSet, buildHierarchy, limit);
+    return new ChildDocTransformer(field, parentsFilter, childDocSet, childSolrReturnFields, buildHierarchy, limit);
   }
 
   private static Query parseQuery(String qstr, SolrQueryRequest req, String param) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e4f256be/solr/core/src/java/org/apache/solr/response/transform/DocTransformer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/response/transform/DocTransformer.java b/solr/core/src/java/org/apache/solr/response/transform/DocTransformer.java
index 7665acd..f1f6bf3 100644
--- a/solr/core/src/java/org/apache/solr/response/transform/DocTransformer.java
+++ b/solr/core/src/java/org/apache/solr/response/transform/DocTransformer.java
@@ -114,4 +114,27 @@ public abstract class DocTransformer {
   public String toString() {
     return getName();
   }
+
+  /**
+   * Trivial Impl that ensure that the specified field is requested as an "extra" field,
+   * but then does nothing during the transformation phase.
+   */
+  public static final class NoopFieldTransformer extends DocTransformer {
+    final String field;
+
+    public NoopFieldTransformer() {
+      this.field = null;
+    }
+
+    public NoopFieldTransformer(String field ) {
+      this.field = field;
+    }
+    public String getName() { return "noop"; }
+    public String[] getExtraRequestFields() {
+      return this.field==null? null: new String[] { field };
+    }
+    public void transform(SolrDocument doc, int docid) {
+      // No-Op
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e4f256be/solr/core/src/java/org/apache/solr/response/transform/RawValueTransformerFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/response/transform/RawValueTransformerFactory.java b/solr/core/src/java/org/apache/solr/response/transform/RawValueTransformerFactory.java
index 25fe56c..55216e5 100644
--- a/solr/core/src/java/org/apache/solr/response/transform/RawValueTransformerFactory.java
+++ b/solr/core/src/java/org/apache/solr/response/transform/RawValueTransformerFactory.java
@@ -84,28 +84,10 @@ public class RawValueTransformerFactory extends TransformerFactory
     
     if (field.equals(display)) {
       // we have to ensure the field is returned
-      return new NoopFieldTransformer(field);
+      return new DocTransformer.NoopFieldTransformer(field);
     }
     return new RenameFieldTransformer( field, display, false );
   }
-
-  /** 
-   * Trivial Impl that ensure that the specified field is requested as an "extra" field, 
-   * but then does nothing during the transformation phase. 
-   */
-  private static final class NoopFieldTransformer extends DocTransformer {
-    final String field;
-    public NoopFieldTransformer(String field ) {
-      this.field = field;
-    }
-    public String getName() { return "noop"; }
-    public String[] getExtraRequestFields() {
-      return new String[] { field };
-    }
-    public void transform(SolrDocument doc, int docid) {
-      // No-Op
-    }
-  }
   
   static class RawTransformer extends DocTransformer
   {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e4f256be/solr/core/src/test/org/apache/solr/response/transform/TestChildDocTransformer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/response/transform/TestChildDocTransformer.java b/solr/core/src/test/org/apache/solr/response/transform/TestChildDocTransformer.java
index 385ca4f..85b476a 100644
--- a/solr/core/src/test/org/apache/solr/response/transform/TestChildDocTransformer.java
+++ b/solr/core/src/test/org/apache/solr/response/transform/TestChildDocTransformer.java
@@ -16,9 +16,15 @@
  */
 package org.apache.solr.response.transform;
 
+import java.util.Collection;
+import java.util.Iterator;
+
 import org.apache.lucene.util.TestUtil;
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.BasicResultContext;
 import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -61,6 +67,7 @@ public class TestChildDocTransformer extends SolrTestCaseJ4 {
     testSubQueryJSON();
 
     testChildDocNonStoredDVFields();
+    testChildReturnFields();
   }
 
   private void testChildDoctransformerXML() throws Exception {
@@ -91,10 +98,10 @@ public class TestChildDocTransformer extends SolrTestCaseJ4 {
         "fl", "*,[child parentFilter=\"subject:parentDocument\"]"), test1);
 
     assertQ(req("q", "*:*", "fq", "subject:\"parentDocument\" ",
-        "fl", "subject,[child parentFilter=\"subject:parentDocument\" childFilter=\"title:foo\"]"), test2);
+        "fl", "id, subject,[child parentFilter=\"subject:parentDocument\" childFilter=\"title:foo\"]"), test2);
 
     assertQ(req("q", "*:*", "fq", "subject:\"parentDocument\" ",
-        "fl", "subject,[child parentFilter=\"subject:parentDocument\" childFilter=\"title:bar\" limit=2]"), test3);
+        "fl", "id, subject,[child parentFilter=\"subject:parentDocument\" childFilter=\"title:bar\" limit=2]"), test3);
   }
   
   private void testSubQueryXML() {
@@ -212,10 +219,10 @@ public class TestChildDocTransformer extends SolrTestCaseJ4 {
         "fl", "*,[child parentFilter=\"subject:parentDocument\"]"), test1);
 
     assertJQ(req("q", "*:*", "fq", "subject:\"parentDocument\" ",
-        "fl", "subject,[child parentFilter=\"subject:parentDocument\" childFilter=\"title:foo\"]"), test2);
+        "fl", "id, subject,[child parentFilter=\"subject:parentDocument\" childFilter=\"title:foo\"]"), test2);
 
     assertJQ(req("q", "*:*", "fq", "subject:\"parentDocument\" ",
-        "fl", "subject,[child parentFilter=\"subject:parentDocument\" childFilter=\"title:bar\" limit=3]"), test3);
+        "fl", "id, subject,[child parentFilter=\"subject:parentDocument\" childFilter=\"title:bar\" limit=3]"), test3);
   }
 
   private void testChildDocNonStoredDVFields() throws Exception {
@@ -243,11 +250,39 @@ public class TestChildDocTransformer extends SolrTestCaseJ4 {
         "fl", "*,[child parentFilter=\"subject:parentDocument\"]"), test1);
 
     assertJQ(req("q", "*:*", "fq", "subject:\"parentDocument\" ",
-        "fl", "subject,[child parentFilter=\"subject:parentDocument\" childFilter=\"title:foo\"]"), test2);
+        "fl", "intDvoDefault, subject,[child parentFilter=\"subject:parentDocument\" childFilter=\"title:foo\"]"), test2);
 
     assertJQ(req("q", "*:*", "fq", "subject:\"parentDocument\" ",
-        "fl", "subject,[child parentFilter=\"subject:parentDocument\" childFilter=\"title:bar\" limit=2]"), test3);
+        "fl", "intDvoDefault, subject,[child parentFilter=\"subject:parentDocument\" childFilter=\"title:bar\" limit=2]"), test3);
+
+  }
+
+  private void testChildReturnFields() throws Exception {
 
+    assertJQ(req("q", "*:*", "fq", "subject:\"parentDocument\" ",
+        "fl", "*,[child parentFilter=\"subject:parentDocument\" fl=\"intDvoDefault,child_fl:[value v='child_fl_test']\"]"),
+        "/response/docs/[0]/intDefault==42",
+        "/response/docs/[0]/_childDocuments_/[0]/intDvoDefault==42",
+        "/response/docs/[0]/_childDocuments_/[0]/child_fl=='child_fl_test'");
+
+    try(SolrQueryRequest req = req("q", "*:*", "fq", "subject:\"parentDocument\" ",
+        "fl", "intDefault,[child parentFilter=\"subject:parentDocument\" fl=\"intDvoDefault, [docid]\"]")) {
+      BasicResultContext res = (BasicResultContext) h.queryAndResponse("/select", req).getResponse();
+      Iterator<SolrDocument> docsStreamer = res.getProcessedDocuments();
+      while (docsStreamer.hasNext()) {
+        SolrDocument doc = docsStreamer.next();
+        assertFalse("root docs should not contain fields specified in child return fields", doc.containsKey("intDvoDefault"));
+        assertTrue("root docs should contain fields specified in query return fields", doc.containsKey("intDefault"));
+        Collection<SolrDocument> childDocs = doc.getChildDocuments();
+        for(SolrDocument childDoc: childDocs) {
+          assertEquals("child doc should only have 2 keys", 2, childDoc.keySet().size());
+          assertTrue("child docs should contain fields specified in child return fields", childDoc.containsKey("intDvoDefault"));
+          assertEquals("child docs should contain fields specified in child return fields",
+              42, childDoc.getFieldValue("intDvoDefault"));
+          assertTrue("child docs should contain fields specified in child return fields", childDoc.containsKey("[docid]"));
+        }
+      }
+    }
   }
 
   private void createSimpleIndex() {
@@ -339,7 +374,7 @@ public class TestChildDocTransformer extends SolrTestCaseJ4 {
     assertJQ(req("q", "*:*", 
                  "sort", "id asc",
                  "fq", "subject:\"parentDocument\" ",
-                 "fl", "id,[child childFilter='cat:childDocument' parentFilter=\"subject:parentDocument\"]"),
+                 "fl", "id, cat, title, [child childFilter='cat:childDocument' parentFilter=\"subject:parentDocument\"]"),
              tests);
 
   }
@@ -398,7 +433,7 @@ public class TestChildDocTransformer extends SolrTestCaseJ4 {
     assertQ(req("q", "*:*", 
                 "sort", "id asc",
                 "fq", "subject:\"parentDocument\" ",
-                "fl", "id,[child childFilter='cat:childDocument' parentFilter=\"subject:parentDocument\"]"),
+                "fl", "id, cat, title, [child childFilter='cat:childDocument' parentFilter=\"subject:parentDocument\"]"),
             tests);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e4f256be/solr/solr-ref-guide/src/transforming-result-documents.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/transforming-result-documents.adoc b/solr/solr-ref-guide/src/transforming-result-documents.adoc
index f9fe805..df9bff4 100644
--- a/solr/solr-ref-guide/src/transforming-result-documents.adoc
+++ b/solr/solr-ref-guide/src/transforming-result-documents.adoc
@@ -137,6 +137,7 @@ When using this transformer, the `parentFilter` parameter must be specified, and
 
 * `childFilter` - query to filter which child documents should be included, this can be particularly useful when you have multiple levels of hierarchical documents (default: all children)
 * `limit` - the maximum number of child documents to be returned per parent document (default: 10)
+* `fl` - the field list which the transformer is to return (default: top level "fl"). There is a further limitation in which the fields here should be a subset of those specified by the top level fl param
 
 
 === [shard] - ShardAugmenterFactory

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e4f256be/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java
index f6628da..1dabe5d 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java
@@ -1831,7 +1831,7 @@ abstract public class SolrExampleTests extends SolrExampleTestsBase
       
       q = new SolrQuery("q", "*:*", "indent", "true");
       q.setFilterQueries(parentFilter);
-      q.setFields("id,[child parentFilter=\"" + parentFilter +
+      q.setFields("id, level_i, [child parentFilter=\"" + parentFilter +
                   "\" childFilter=\"" + childFilter + 
                   "\" limit=\"" + maxKidCount + "\"], name");
       resp = client.query(q);
@@ -1916,7 +1916,7 @@ abstract public class SolrExampleTests extends SolrExampleTestsBase
         q = new SolrQuery("q", "name:" + name, "indent", "true");
       }
       q.setFilterQueries(parentFilter);
-      q.setFields("id,[child parentFilter=\"" + parentFilter +
+      q.setFields("id, level_i, [child parentFilter=\"" + parentFilter +
                   "\" childFilter=\"" + childFilter + 
                   "\" limit=\"" + maxKidCount + "\"],name");
       resp = client.query(q);