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 2019/01/28 18:25:21 UTC

[lucene-solr] branch master updated: SOLR-12768: added _nest_path_ to the default schema (thereby enabling nested docs) * new NestPathField encapsulating details for how _nest_path_ is indexed ** tweaked the analysis to index 1 token instead of variable * TokenizerChain has new CustomAnalyzer copy-constructor

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 381a30b  SOLR-12768: added _nest_path_ to the default schema (thereby enabling nested docs) * new NestPathField encapsulating details for how _nest_path_ is indexed ** tweaked the analysis to index 1 token instead of variable * TokenizerChain has new CustomAnalyzer copy-constructor
381a30b is described below

commit 381a30b26ca1737123b65aefc685367d1aa038b9
Author: David Smiley <ds...@apache.org>
AuthorDate: Mon Jan 28 13:25:06 2019 -0500

    SOLR-12768: added _nest_path_ to the default schema (thereby enabling nested docs)
    * new NestPathField encapsulating details for how _nest_path_ is indexed
    ** tweaked the analysis to index 1 token instead of variable
    * TokenizerChain has new CustomAnalyzer copy-constructor
---
 solr/CHANGES.txt                                   |  7 +++
 .../org/apache/solr/analysis/TokenizerChain.java   | 13 +++++
 .../transform/ChildDocTransformerFactory.java      | 15 +++--
 .../src/java/org/apache/solr/schema/FieldType.java |  5 +-
 .../java/org/apache/solr/schema/NestPathField.java | 66 ++++++++++++++++++++++
 .../processor/NestedUpdateProcessorFactory.java    | 10 ++--
 .../solr/collection1/conf/schema-nest.xml          | 20 ++-----
 .../solr/configsets/_default/conf/managed-schema   |  6 ++
 .../TestChildDocTransformerHierarchy.java          | 43 +++++++++++---
 .../solr/update/TestNestedUpdateProcessor.java     | 37 ++++++++----
 .../solr/configsets/_default/conf/managed-schema   |  6 ++
 11 files changed, 182 insertions(+), 46 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 169081d..9fae7ad 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -149,6 +149,13 @@ New Features
 
 * SOLR-12639: Umbrella JIRA for adding support HTTP/2 (Cao Manh Dat)
 
+* SOLR-12768: Improved nested document support, and enabled in the default schema with the presence of _nest_path_.
+  When this field is present, certain things happen automatically.  An internal URP is automatically used to populate
+  it.  The [child] (doc transformer) will return a hierarchy with relationships; no params needed.  The relationship
+  path is indexed for use in queries (can be disabled if not needed).  Also, child documents needn't provide a uniqueKey
+  value as Solr will supply one automatically by concatenating a path to that of the parent document's key.
+  (David Smiley, Moshe Bla).
+
 Bug Fixes
 ----------------------
 
diff --git a/solr/core/src/java/org/apache/solr/analysis/TokenizerChain.java b/solr/core/src/java/org/apache/solr/analysis/TokenizerChain.java
index 95755e1..9938045 100644
--- a/solr/core/src/java/org/apache/solr/analysis/TokenizerChain.java
+++ b/solr/core/src/java/org/apache/solr/analysis/TokenizerChain.java
@@ -22,6 +22,7 @@ import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.core.KeywordTokenizer;
+import org.apache.lucene.analysis.custom.CustomAnalyzer;
 import org.apache.lucene.analysis.util.CharFilterFactory;
 import org.apache.lucene.analysis.util.TokenFilterFactory;
 import org.apache.lucene.analysis.util.TokenizerFactory;
@@ -30,6 +31,7 @@ import org.apache.lucene.analysis.util.TokenizerFactory;
  * An analyzer that uses a tokenizer and a list of token filters to
  * create a TokenStream.
  *
+ * It should probably be replaced with {@link CustomAnalyzer}.
  * @since 3.1
  */
 public final class TokenizerChain extends SolrAnalyzer {
@@ -40,6 +42,17 @@ public final class TokenizerChain extends SolrAnalyzer {
   final private TokenizerFactory tokenizer;
   final private TokenFilterFactory[] filters;
 
+  /** Copy from CustomAnalyzer. */
+  public TokenizerChain(CustomAnalyzer customAnalyzer) {
+    this(
+        customAnalyzer.getCharFilterFactories().toArray(new CharFilterFactory[0]),
+        customAnalyzer.getTokenizerFactory(),
+        customAnalyzer.getTokenFilterFactories().toArray(new TokenFilterFactory[0]));
+    setPositionIncrementGap(customAnalyzer.getPositionIncrementGap(null));
+    setVersion(customAnalyzer.getVersion());
+    assert customAnalyzer.getOffsetGap(null) == 1; // note: we don't support setting the offset gap
+  }
+
   /** 
    * Creates a new TokenizerChain w/o any CharFilterFactories.
    *
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 82be49d..b38565e 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
@@ -150,19 +150,24 @@ public class ChildDocTransformerFactory extends TransformerFactory {
   // NOTE: THIS FEATURE IS PRESENTLY EXPERIMENTAL; WAIT TO SEE IT IN THE REF GUIDE.  FINAL SYNTAX IS TBD.
   protected static String processPathHierarchyQueryString(String queryString) {
     // if the filter includes a path string, build a lucene query string to match those specific child documents.
-    // e.g. toppings/ingredients/name_s:cocoa -> +_nest_path_:"toppings/ingredients/" +(name_s:cocoa)
+    // e.g. /toppings/ingredients/name_s:cocoa -> +_nest_path_:/toppings/ingredients +(name_s:cocoa)
+    // ingredients/name_s:cocoa -> +_nest_path_:*/ingredients +(name_s:cocoa)
     int indexOfFirstColon = queryString.indexOf(':');
     if (indexOfFirstColon <= 0) {
       return queryString;// give up
     }
     int indexOfLastPathSepChar = queryString.lastIndexOf(PATH_SEP_CHAR, indexOfFirstColon);
     if (indexOfLastPathSepChar < 0) {
-      return queryString;
+      // regular filter, not hierarchy based.
+      return ClientUtils.escapeQueryChars(queryString.substring(0, indexOfFirstColon))
+          + ":" + ClientUtils.escapeQueryChars(queryString.substring(indexOfFirstColon + 1));
     }
-    String path = queryString.substring(0, indexOfLastPathSepChar + 1);
-    String remaining = queryString.substring(indexOfLastPathSepChar + 1);
+    final boolean isAbsolutePath = queryString.charAt(0) == PATH_SEP_CHAR;
+    String path = ClientUtils.escapeQueryChars(queryString.substring(0, indexOfLastPathSepChar));
+    String remaining = queryString.substring(indexOfLastPathSepChar + 1); // last part of path hierarchy
+
     return
-        "+" + NEST_PATH_FIELD_NAME + ":" + ClientUtils.escapeQueryChars(path)
+        "+" + NEST_PATH_FIELD_NAME + (isAbsolutePath? ":": ":*\\/") + path
         + " +(" + remaining + ")";
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/schema/FieldType.java b/solr/core/src/java/org/apache/solr/schema/FieldType.java
index 8bcf839..f960d8e 100644
--- a/solr/core/src/java/org/apache/solr/schema/FieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/FieldType.java
@@ -164,7 +164,10 @@ public abstract class FieldType extends FieldProperties {
     return false;
   }
 
-  // Handle additional arguments...
+  /**
+   * Initializes the field type.  Subclasses should usually override {@link #init(IndexSchema, Map)}
+   * which is called by this method.
+   */
   protected void setArgs(IndexSchema schema, Map<String,String> args) {
     // default to STORED, INDEXED, OMIT_TF_POSITIONS and MULTIVALUED depending on schema version
     properties = (STORED | INDEXED);
diff --git a/solr/core/src/java/org/apache/solr/schema/NestPathField.java b/solr/core/src/java/org/apache/solr/schema/NestPathField.java
new file mode 100644
index 0000000..926aa7e
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/schema/NestPathField.java
@@ -0,0 +1,66 @@
+/*
+ * 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.schema;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.lucene.analysis.core.KeywordTokenizerFactory;
+import org.apache.lucene.analysis.custom.CustomAnalyzer;
+import org.apache.lucene.analysis.pattern.PatternReplaceFilterFactory;
+import org.apache.solr.analysis.TokenizerChain;
+import org.apache.solr.common.SolrException;
+
+/**
+ * To be used for field {@link IndexSchema#NEST_PATH_FIELD_NAME} for enhanced
+ * nested doc information.  By defining a field type, we can encapsulate the configuration
+ * here so that the schema is free of it.  Alternatively, some notion of "implicit field types"
+ * would be cool and a more general way of accomplishing this.
+ *
+ * @see org.apache.solr.update.processor.NestedUpdateProcessorFactory
+ * @since 8.0
+ */
+public class NestPathField extends SortableTextField {
+
+  @Override
+  public void setArgs(IndexSchema schema, Map<String, String> args) {
+    args.putIfAbsent("stored", "false");
+    args.putIfAbsent("omitTermFreqAndPositions", "true");
+    args.putIfAbsent("omitNorms", "true");
+    args.putIfAbsent("maxCharsForDocValues", "-1");
+    super.setArgs(schema, args);
+
+    // CustomAnalyzer is easy to use
+    CustomAnalyzer customAnalyzer;
+    try {
+      customAnalyzer = CustomAnalyzer.builder(schema.getResourceLoader())
+          .withDefaultMatchVersion(schema.getDefaultLuceneMatchVersion())
+          .withTokenizer(KeywordTokenizerFactory.class)
+          .addTokenFilter(PatternReplaceFilterFactory.class,
+              "pattern", "#\\d*",
+              "replace", "all")
+          .build();
+    } catch (IOException e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);//impossible?
+    }
+    // Solr HTTP Schema APIs don't know about CustomAnalyzer so use TokenizerChain instead
+    setIndexAnalyzer(new TokenizerChain(customAnalyzer));
+    // leave queryAnalyzer as literal
+  }
+
+}
diff --git a/solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessorFactory.java b/solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessorFactory.java
index aa459bd..af109f7 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessorFactory.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessorFactory.java
@@ -45,7 +45,7 @@ public class NestedUpdateProcessorFactory extends UpdateRequestProcessorFactory
     if(!(storeParent || storePath)) {
       return next;
     }
-    return new NestedUpdateProcessor(req, shouldStoreDocParent(req.getSchema()), shouldStoreDocPath(req.getSchema()), next);
+    return new NestedUpdateProcessor(req, storeParent, storePath, next);
   }
 
   private static boolean shouldStoreDocParent(IndexSchema schema) {
@@ -100,10 +100,10 @@ public class NestedUpdateProcessorFactory extends UpdateRequestProcessorFactory
             String parentDocId = doc.getField(uniqueKeyFieldName).getFirstValue().toString();
             cDoc.setField(uniqueKeyFieldName, generateChildUniqueId(parentDocId, fieldName, sChildNum));
           }
-          final String lastKeyPath = fieldName + NUM_SEP_CHAR + sChildNum;
-          // concat of all paths children.grandChild => children#1/grandChild#
-          final String childDocPath = fullPath == null ? lastKeyPath : fullPath + PATH_SEP_CHAR + lastKeyPath;
-          processChildDoc((SolrInputDocument) val, doc, childDocPath);
+          final String lastKeyPath = PATH_SEP_CHAR + fieldName + NUM_SEP_CHAR + sChildNum;
+          // concat of all paths children.grandChild => /children#1/grandChild#
+          final String childDocPath = fullPath == null ? lastKeyPath : fullPath + lastKeyPath;
+          processChildDoc(cDoc, doc, childDocPath);
           ++childNum;
         }
       }
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-nest.xml b/solr/core/src/test-files/solr/collection1/conf/schema-nest.xml
index 313e586..d20d734 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema-nest.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema-nest.xml
@@ -29,9 +29,9 @@
   <!-- points to the root document of a block of nested documents -->
   <field name="_root_" type="string" indexed="true" stored="true"/>
 
-  <!-- required for NestedUpdateProcessor -->
+  <!-- populated by for NestedUpdateProcessor -->
   <field name="_nest_parent_" type="string" indexed="true" stored="true"/>
-  <field name="_nest_path_" type="descendants_path" indexed="true" multiValued="false" docValues="true" stored="false" useDocValuesAsStored="false"/>
+  <field name="_nest_path_" type="_nest_path_" />
 
   <dynamicField name="*_s" type="string" indexed="true" stored="true"/>
   <dynamicField name="*_ss" type="string" indexed="true" stored="true" multiValued="true"/>
@@ -39,6 +39,8 @@
 
   <fieldType name="string" class="solr.StrField" sortMissingLast="true"/>
 
+  <fieldType name="_nest_path_" class="solr.NestPathField" />
+
   <!-- Point Fields -->
   <fieldType name="int" class="solr.IntPointField" docValues="true"/>
   <fieldType name="long" class="solr.LongPointField" docValues="true"/>
@@ -46,20 +48,6 @@
   <fieldType name="float" class="solr.FloatPointField" docValues="true"/>
   <fieldType name="date" class="solr.DatePointField" docValues="true"/>
 
-  <fieldType name="descendants_path" class="solr.SortableTextField">
-    <analyzer type="index">
-      <!--char filter to append / to path in the indexed form e.g. toppings/ingredients turns to toppings/ingredients/ -->
-      <charFilter class="solr.PatternReplaceCharFilterFactory" pattern="(^.*.*$)" replacement="$0/"/>
-      <!--tokenize the path so path queries are optimized -->
-      <tokenizer class="solr.PathHierarchyTokenizerFactory" delimiter="/"/>
-      <!--remove the # and digit index of array from path toppings#1/ingredients#/ turns to toppings/ingredients/ -->
-      <filter class="solr.PatternReplaceFilterFactory" pattern="[#*\d]*" replace="all"/>
-    </analyzer>
-    <analyzer type="query">
-      <tokenizer class="solr.KeywordTokenizerFactory"/>
-    </analyzer>
-  </fieldType>
-
   <uniqueKey>id</uniqueKey>
 
 </schema>
diff --git a/solr/core/src/test-files/solr/configsets/_default/conf/managed-schema b/solr/core/src/test-files/solr/configsets/_default/conf/managed-schema
index b2e3272..5b9b9bf 100644
--- a/solr/core/src/test-files/solr/configsets/_default/conf/managed-schema
+++ b/solr/core/src/test-files/solr/configsets/_default/conf/managed-schema
@@ -113,7 +113,13 @@
     <field name="id" type="string" indexed="true" stored="true" required="true" multiValued="false" />
     <!-- docValues are enabled by default for long type so we don't need to index the version field  -->
     <field name="_version_" type="plong" indexed="false" stored="false"/>
+
+    <!-- If you don't use child/nested documents, then you should remove the next two fields:  -->
+    <!-- for nested documents (minimal; points to root document) -->
     <field name="_root_" type="string" indexed="true" stored="false" docValues="false" />
+    <!-- for nested documents (relationship tracking) -->
+    <field name="_nest_path_" type="_nest_path_" /><fieldType name="_nest_path_" class="solr.NestPathField" />
+
     <field name="_text_" type="text_general" indexed="true" stored="false" multiValued="true"/>
 
     <!-- This can be enabled, in case the client does not know what fields may be searched. It isn't enabled by default
diff --git a/solr/core/src/test/org/apache/solr/response/transform/TestChildDocTransformerHierarchy.java b/solr/core/src/test/org/apache/solr/response/transform/TestChildDocTransformerHierarchy.java
index e37f31d..26e5f05 100644
--- a/solr/core/src/test/org/apache/solr/response/transform/TestChildDocTransformerHierarchy.java
+++ b/solr/core/src/test/org/apache/solr/response/transform/TestChildDocTransformerHierarchy.java
@@ -111,7 +111,7 @@ public class TestChildDocTransformerHierarchy extends SolrTestCaseJ4 {
   public void testParentFilterLimitJSON() throws Exception {
     indexSampleData(numberOfDocsPerNestedTest);
 
-    try(SolrQueryRequest req = req("q", "type_s:donut", "sort", "id asc", "fl", "id, type_s, toppings, _nest_path_, [child childFilter='_nest_path_:\"toppings/\"' limit=1]",
+    try(SolrQueryRequest req = req("q", "type_s:donut", "sort", "id asc", "fl", "id, type_s, toppings, _nest_path_, [child childFilter='_nest_path_:/toppings' limit=1]",
         "fq", fqToExcludeNonTestedDocs)) {
       BasicResultContext res = (BasicResultContext) h.queryAndResponse("/select", req).getResponse();
       Iterator<SolrDocument> docsStreamer = res.getProcessedDocuments();
@@ -165,19 +165,19 @@ public class TestChildDocTransformerHierarchy extends SolrTestCaseJ4 {
     indexSampleData(2);
     String[] tests = {
         "/response/numFound==4",
-        "/response/docs/[0]/_nest_path_=='toppings#0'",
-        "/response/docs/[1]/_nest_path_=='toppings#0'",
-        "/response/docs/[2]/_nest_path_=='toppings#1'",
-        "/response/docs/[3]/_nest_path_=='toppings#1'",
+        "/response/docs/[0]/_nest_path_=='/toppings#0'",
+        "/response/docs/[1]/_nest_path_=='/toppings#0'",
+        "/response/docs/[2]/_nest_path_=='/toppings#1'",
+        "/response/docs/[3]/_nest_path_=='/toppings#1'",
     };
 
-    assertJQ(req("q", "_nest_path_:*toppings/",
+    assertJQ(req("q", "_nest_path_:*toppings",
         "sort", "_nest_path_ asc",
         "fl", "*, id_i, _nest_path_",
         "fq", fqToExcludeNonTestedDocs),
         tests);
 
-    assertJQ(req("q", "+_nest_path_:\"toppings/\"",
+    assertJQ(req("q", "+_nest_path_:\"/toppings\"",
         "sort", "_nest_path_ asc",
         "fl", "*, _nest_path_",
         "fq", fqToExcludeNonTestedDocs),
@@ -219,16 +219,43 @@ public class TestChildDocTransformerHierarchy extends SolrTestCaseJ4 {
       }
     }
 
+    // test full path
+    assertJQ(req("q", "type_s:donut",
+        "sort", "id asc",
+        "fl", "*,[child childFilter='toppings/ingredients/name_s:cocoa']",
+        "fq", fqToExcludeNonTestedDocs),
+        tests);
 
+    // test partial path
+    assertJQ(req("q", "type_s:donut",
+        "sort", "id asc",
+        "fl", "*,[child childFilter='ingredients/name_s:cocoa']",
+        "fq", fqToExcludeNonTestedDocs),
+        tests);
 
+    // test absolute path
     assertJQ(req("q", "type_s:donut",
         "sort", "id asc",
-        "fl", "*,[child childFilter='toppings/ingredients/name_s:cocoa']",
+        "fl", "*,[child childFilter='/toppings/ingredients/name_s:cocoa']",
         "fq", fqToExcludeNonTestedDocs),
         tests);
   }
 
   @Test
+  public void testNestPathTransformerMatches() throws Exception {
+    indexSampleData(numberOfDocsPerNestedTest);
+
+    // test partial path
+    // should not match any child docs
+    assertQ(req("q", "type_s:donut",
+        "sort", "id asc",
+        "fl", "*,[child childFilter='redients/name_s:cocoa']",
+        "fq", fqToExcludeNonTestedDocs),
+        "//result/doc/str[@name='type_s'][.='donut']", "not(//result/doc/arr[@name='toppings'])"
+        );
+  }
+
+  @Test
   public void testSingularChildFilterJSON() throws Exception {
     indexSampleData(numberOfDocsPerNestedTest);
     String[] tests = new String[] {
diff --git a/solr/core/src/test/org/apache/solr/update/TestNestedUpdateProcessor.java b/solr/core/src/test/org/apache/solr/update/TestNestedUpdateProcessor.java
index a1aa546..4384895 100644
--- a/solr/core/src/test/org/apache/solr/update/TestNestedUpdateProcessor.java
+++ b/solr/core/src/test/org/apache/solr/update/TestNestedUpdateProcessor.java
@@ -95,7 +95,7 @@ public class TestNestedUpdateProcessor extends SolrTestCaseJ4 {
 
   @Before
   public void before() throws Exception {
-    assertU(delQ("*:*"));
+    clearIndex();
     assertU(commit());
   }
 
@@ -103,7 +103,7 @@ public class TestNestedUpdateProcessor extends SolrTestCaseJ4 {
   public void testDeeplyNestedURPGrandChild() throws Exception {
     final String[] tests = {
         "/response/docs/[0]/id=='4'",
-        "/response/docs/[0]/" + IndexSchema.NEST_PATH_FIELD_NAME + "=='children#0/grandChild#'"
+        "/response/docs/[0]/" + IndexSchema.NEST_PATH_FIELD_NAME + "=='/children#0/grandChild#'"
     };
     indexSampleData(jDoc);
 
@@ -115,27 +115,42 @@ public class TestNestedUpdateProcessor extends SolrTestCaseJ4 {
   }
 
   @Test
+  public void testNumberInName() throws Exception {
+    // child named "grandChild99"  (has a number in it)
+    indexSampleData(jDoc.replace("grandChild", "grandChild99"));
+    //assertQ(req("qt", "/terms", "terms", "true", "terms.fl", IndexSchema.NEST_PATH_FIELD_NAME), "false"); // for debugging
+
+    // find it
+    assertJQ(req("q", "{!field f=" + IndexSchema.NEST_PATH_FIELD_NAME + "}/children/grandChild99"),
+        "/response/numFound==1");
+    // should *NOT* find it; different number
+    assertJQ(req("q", "{!field f=" + IndexSchema.NEST_PATH_FIELD_NAME + "}/children/grandChild22"),
+        "/response/numFound==0");
+
+  }
+
+  @Test
   public void testDeeplyNestedURPChildren() throws Exception {
     final String[] childrenTests = {
         "/response/docs/[0]/id=='2'",
         "/response/docs/[1]/id=='3'",
-        "/response/docs/[0]/" + IndexSchema.NEST_PATH_FIELD_NAME + "=='children#0'",
-        "/response/docs/[1]/" + IndexSchema.NEST_PATH_FIELD_NAME + "=='children#1'"
+        "/response/docs/[0]/" + IndexSchema.NEST_PATH_FIELD_NAME + "=='/children#0'",
+        "/response/docs/[1]/" + IndexSchema.NEST_PATH_FIELD_NAME + "=='/children#1'"
     };
     indexSampleData(jDoc);
 
-    assertJQ(req("q", IndexSchema.NEST_PATH_FIELD_NAME + ":children/",
+    assertJQ(req("q", IndexSchema.NEST_PATH_FIELD_NAME + ":\\/children",
         "fl","*, _nest_path_",
         "sort","id asc",
         "wt","json"),
         childrenTests);
 
-    assertJQ(req("q", IndexSchema.NEST_PATH_FIELD_NAME + ":anotherChildList/",
+    assertJQ(req("q", IndexSchema.NEST_PATH_FIELD_NAME + ":\\/anotherChildList",
         "fl","*, _nest_path_",
         "sort","id asc",
         "wt","json"),
         "/response/docs/[0]/id=='4'",
-        "/response/docs/[0]/" + IndexSchema.NEST_PATH_FIELD_NAME + "=='anotherChildList#0'");
+        "/response/docs/[0]/" + IndexSchema.NEST_PATH_FIELD_NAME + "=='/anotherChildList#0'");
   }
 
   @Test
@@ -151,16 +166,16 @@ public class TestNestedUpdateProcessor extends SolrTestCaseJ4 {
     List children = (List) docHierarchy.get("children").getValues();
 
     SolrInputDocument firstChild = (SolrInputDocument) children.get(0);
-    assertEquals("SolrInputDocument(fields: [id=2, name_s=Yaz, _nest_path_=children#0, _nest_parent_=1])", firstChild.toString());
+    assertEquals("SolrInputDocument(fields: [id=2, name_s=Yaz, _nest_path_=/children#0, _nest_parent_=1])", firstChild.toString());
 
     SolrInputDocument secondChild = (SolrInputDocument) children.get(1);
-    assertEquals("SolrInputDocument(fields: [id=3, name_s=Jazz, grandChild=SolrInputDocument(fields: [id=4, name_s=Gaz, _nest_path_=children#1/grandChild#, _nest_parent_=3]), _nest_path_=children#1, _nest_parent_=1])", secondChild.toString());
+    assertEquals("SolrInputDocument(fields: [id=3, name_s=Jazz, grandChild=SolrInputDocument(fields: [id=4, name_s=Gaz, _nest_path_=/children#1/grandChild#, _nest_parent_=3]), _nest_path_=/children#1, _nest_parent_=1])", secondChild.toString());
 
     SolrInputDocument grandChild = (SolrInputDocument)((SolrInputDocument) children.get(1)).get("grandChild").getValue();
-    assertEquals("SolrInputDocument(fields: [id=4, name_s=Gaz, _nest_path_=children#1/grandChild#, _nest_parent_=3])", grandChild.toString());
+    assertEquals("SolrInputDocument(fields: [id=4, name_s=Gaz, _nest_path_=/children#1/grandChild#, _nest_parent_=3])", grandChild.toString());
 
     SolrInputDocument singularChild = (SolrInputDocument) docHierarchy.get("lonelyChild").getValue();
-    assertEquals("SolrInputDocument(fields: [id=5, name_s=Loner, _nest_path_=lonelyChild#, _nest_parent_=1])", singularChild.toString());
+    assertEquals("SolrInputDocument(fields: [id=5, name_s=Loner, _nest_path_=/lonelyChild#, _nest_parent_=1])", singularChild.toString());
   }
 
   @Test
diff --git a/solr/server/solr/configsets/_default/conf/managed-schema b/solr/server/solr/configsets/_default/conf/managed-schema
index b2e3272..5b9b9bf 100644
--- a/solr/server/solr/configsets/_default/conf/managed-schema
+++ b/solr/server/solr/configsets/_default/conf/managed-schema
@@ -113,7 +113,13 @@
     <field name="id" type="string" indexed="true" stored="true" required="true" multiValued="false" />
     <!-- docValues are enabled by default for long type so we don't need to index the version field  -->
     <field name="_version_" type="plong" indexed="false" stored="false"/>
+
+    <!-- If you don't use child/nested documents, then you should remove the next two fields:  -->
+    <!-- for nested documents (minimal; points to root document) -->
     <field name="_root_" type="string" indexed="true" stored="false" docValues="false" />
+    <!-- for nested documents (relationship tracking) -->
+    <field name="_nest_path_" type="_nest_path_" /><fieldType name="_nest_path_" class="solr.NestPathField" />
+
     <field name="_text_" type="text_general" indexed="true" stored="false" multiValued="true"/>
 
     <!-- This can be enabled, in case the client does not know what fields may be searched. It isn't enabled by default