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 2021/01/08 04:23:38 UTC

[lucene-solr] branch master updated: * SOLR-14923: Nested docs indexing perf & robustness (#2159)

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 4cb3ad4  * SOLR-14923: Nested docs indexing perf & robustness (#2159)
4cb3ad4 is described below

commit 4cb3ad4a1c40b4326aec64577a7e60018f7f1a5e
Author: David Smiley <ds...@apache.org>
AuthorDate: Thu Jan 7 23:23:20 2021 -0500

    * SOLR-14923: Nested docs indexing perf & robustness (#2159)
    
    * When the schema defines _root_, and you want to do atomic/partial updates...
    ** _root_ needn't be stored or have docValues any more
    ** _nest_path_ field isn't needed for this any more
    ** Simplified internal logic
    * Allow (and recommend, eventually insist) that the _root_ field be passed for atomic/partial updates to child docs.
    ** In the absence of _root_, assume the _route_ param is equivalent to ameliorate back-compat scope.  This is a temporary hack; remove in SOLR-15064.
    ** One of the two is required; you'll get an exception if the assumption is false.  THIS IS A BACK-COMPAT CHANGE
    * Ensure that the update log contains the _root_ field if it's defined in the schema; in some cases it wasn't.  It's important for robustness of atomic/partial updates to child docs.  Caveat: the buffer replay scenario is not tested with child docs.
    * Limited the cases when a realtime searcher is re-opened.  It was being applied to any update that included child docs but now only some narrow subset: only for atomic/partial updates, and when the update log contains an in-place update for the same nest because it's complicated to resolve those log entries.
    * Internal improvements to RealTimeGetComponent to aid clarity & robustness & probably performance...
    ** Use SolrDocumentFetcher.solrDoc(docID, ReturnFields) instead of more manual loading.  Will do more with this in another PR.
    ** Clarify when only root doc IDs are expected.
    ** Use Resolution enum more, add PARTIAL, remove DOC_WITH_CHILDREN; enhance docs.
    ** When have ReturnFields, a Set of "onlyTheseFields" becomes redundant.  Add a child doc resolution via a transformer when needed.
    ** Clarified where copy-field targets are removed
    * NestPathField should default to single valued, instead of inheriting the schema default, which for ancient schemas was multi-valued.
    * AddUpdateCommand.getLuceneDocument(s) methods are very internal; made package visible and refactored a bit for clarity
    * DocumentBuilder: when in-place update, skip id and _root_ here, thus also simplifying further logic
    * NestedShardedAtomicUpdateTest no longer extends AbstractFullDistribZkTestBase because it wasn't really leveraging the "control client" checking, and it added too much complexity to debug failures.
---
 solr/CHANGES.txt                                   |   9 +
 .../handler/component/RealTimeGetComponent.java    | 411 ++++++++++++---------
 .../java/org/apache/solr/schema/IndexSchema.java   |  42 +--
 .../java/org/apache/solr/schema/NestPathField.java |   1 +
 .../org/apache/solr/search/SolrReturnFields.java   |  29 ++
 .../org/apache/solr/update/AddUpdateCommand.java   | 199 ++++++----
 .../apache/solr/update/DirectUpdateHandler2.java   |  47 +--
 .../org/apache/solr/update/DocumentBuilder.java    |  53 +--
 .../src/java/org/apache/solr/update/UpdateLog.java |  12 +
 .../processor/AtomicUpdateDocumentMerger.java      | 126 +++----
 .../processor/ClassificationUpdateProcessor.java   |   6 +-
 .../processor/DistributedUpdateProcessor.java      |  93 ++---
 .../processor/DistributedZkUpdateProcessor.java    |   6 +-
 .../processor/NestedUpdateProcessorFactory.java    |   2 +-
 .../SkipExistingDocumentsProcessorFactory.java     |  17 +-
 .../solr/collection1/conf/schema-nest.xml          |   2 +-
 .../solr/cloud/NestedShardedAtomicUpdateTest.java  | 194 ++++++----
 .../solr/update/processor/AtomicUpdatesTest.java   |  31 --
 .../update/processor/NestedAtomicUpdateTest.java   |  98 +++--
 .../src/indexing-nested-documents.adoc             | 101 +++--
 solr/solr-ref-guide/src/solr-upgrade-notes.adoc    |  14 +
 .../src/updating-parts-of-documents.adoc           |  24 +-
 .../java/org/apache/solr/common/SolrDocument.java  |  28 ++
 .../org/apache/solr/common/SolrInputDocument.java  |  27 ++
 24 files changed, 920 insertions(+), 652 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index f9d5293..0f4f217 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -239,6 +239,11 @@ Improvements
 * SOLR-15069: [child]: the parentFilter parameter is now fully optional and perhaps obsolete.
   (David Smiley)
 
+* SOLR-14923: The use case of atomic/partial updates to child documents no longer requires that
+  _root_ be stored or have docValues, and no longer requires the _nest_path_ field.  However it
+  now requires that the client pass a _root_ field on these updates to point to the root ID.
+  (David Smiley)
+
 * SOLR-15059: Add panels to the default Grafana dashboard for query performance monitoring, includes updates
   to the Prometheus exporter to export query performance metrics, such as QPS and p95. (Timothy Potter)
 
@@ -250,6 +255,10 @@ Optimizations
 
 * SOLR-15049: Optimize same-core, same-field joins in TopLevelJoinQuery (Jason Gerlowski)
 
+* SOLR-14923: Indexing nested documents is faster, especially under concurrent indexing load.  In addition,
+  Partial updates to nested documents and Realtime Get of child documents is now more reliable.
+  (David Smiley, Thomas Wöckinger)
+
 Bug Fixes
 ---------------------
 * SOLR-14946: Fix responseHeader being returned in response when omitHeader=true and EmbeddedSolrServer is used
diff --git a/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java b/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
index d824bd5..ad14837 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
@@ -29,16 +29,19 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.BiConsumer;
 import java.util.stream.Collectors;
 
 import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreMode;
@@ -49,7 +52,6 @@ import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.SolrDocument;
-import org.apache.solr.common.SolrDocumentBase;
 import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
@@ -100,7 +102,6 @@ import static org.apache.solr.common.params.CommonParams.VERSION_FIELD;
 public class RealTimeGetComponent extends SearchComponent
 {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  private static final Set<String> NESTED_META_FIELDS = Sets.newHashSet(IndexSchema.NEST_PATH_FIELD_NAME, IndexSchema.NEST_PARENT_FIELD_NAME);
   public static final String COMPONENT_NAME = "get";
 
   @Override
@@ -239,11 +240,17 @@ public class RealTimeGetComponent extends SearchComponent
 
    try {
 
-
+     boolean opennedRealtimeSearcher = false;
      BytesRefBuilder idBytes = new BytesRefBuilder();
      for (String idStr : reqIds.allIds) {
        fieldType.readableToIndexed(idStr, idBytes);
-       if (ulog != null) {
+       // if _route_ is passed, id is a child doc.  TODO remove in SOLR-15064
+       if (!opennedRealtimeSearcher && !params.get(ShardParams._ROUTE_, idStr).equals(idStr)) {
+         searcherInfo.clear();
+         resultContext = null;
+         ulog.openRealtimeSearcher();  // force open a new realtime searcher
+         opennedRealtimeSearcher = true;
+       } else if (ulog != null) {
          Object o = ulog.lookup(idBytes.get());
          if (o != null) {
            // should currently be a List<Oper,Ver,Doc/Id>
@@ -257,9 +264,12 @@ public class RealTimeGetComponent extends SearchComponent
 
                if (mustUseRealtimeSearcher) {
                  // close handles to current searchers & result context
-                 searcherInfo.clear();
-                 resultContext = null;
-                 ulog.openRealtimeSearcher();  // force open a new realtime searcher
+                 if (!opennedRealtimeSearcher) {
+                   searcherInfo.clear();
+                   resultContext = null;
+                   ulog.openRealtimeSearcher();  // force open a new realtime searcher
+                   opennedRealtimeSearcher = true;
+                 }
                  o = null;  // pretend we never found this record and fall through to use the searcher
                  break;
                }
@@ -267,20 +277,24 @@ public class RealTimeGetComponent extends SearchComponent
                SolrDocument doc;
                if (oper == UpdateLog.ADD) {
                  doc = toSolrDoc((SolrInputDocument)entry.get(entry.size()-1), core.getLatestSchema());
+                 // toSolrDoc filtered copy-field targets already
+                 if (transformer!=null) {
+                   transformer.transform(doc, -1); // unknown docID
+                 }
                } else if (oper == UpdateLog.UPDATE_INPLACE) {
                  assert entry.size() == 5;
                  // For in-place update case, we have obtained the partial document till now. We need to
                  // resolve it to a full document to be returned to the user.
-                 doc = resolveFullDocument(core, idBytes.get(), rsp.getReturnFields(), (SolrInputDocument)entry.get(entry.size()-1), entry, null);
+                 // resolveFullDocument applies the transformer, if present.
+                 doc = resolveFullDocument(core, idBytes.get(), rsp.getReturnFields(), (SolrInputDocument)entry.get(entry.size()-1), entry);
                  if (doc == null) {
                    break; // document has been deleted as the resolve was going on
                  }
+                 doc.visitSelfAndNestedDocs((label, d) -> removeCopyFieldTargets(d, req.getSchema()));
                } else {
                  throw new SolrException(ErrorCode.INVALID_STATE, "Expected ADD or UPDATE_INPLACE. Got: " + oper);
                }
-               if (transformer!=null) {
-                 transformer.transform(doc, -1); // unknown docID
-               }
+
               docList.add(doc);
               break;
              case UpdateLog.DELETE:
@@ -326,12 +340,12 @@ public class RealTimeGetComponent extends SearchComponent
          if (null == resultContext) {
            // either first pass, or we've re-opened searcher - either way now we setContext
            resultContext = new RTGResultContext(rsp.getReturnFields(), searcherInfo.getSearcher(), req);
-           transformer.setContext(resultContext);
+           transformer.setContext(resultContext); // we avoid calling setContext unless searcher is new/changed
          }
          transformer.transform(doc, docid);
        }
        docList.add(doc);
-     }
+     } // loop on ids
 
    } finally {
      searcherInfo.clear();
@@ -342,7 +356,8 @@ public class RealTimeGetComponent extends SearchComponent
   
   /**
    * Return the requested SolrInputDocument from the tlog/index. This will
-   * always be a full document, i.e. any partial in-place document will be resolved.
+   * always be a full document with children; partial / in-place documents will be resolved.
+   * The id must be for a root document, not a child.
    */
   void processGetInputDocument(ResponseBuilder rb) throws IOException {
     SolrQueryRequest req = rb.req;
@@ -355,8 +370,9 @@ public class RealTimeGetComponent extends SearchComponent
 
     String idStr = params.get("getInputDocument", null);
     if (idStr == null) return;
+    BytesRef idBytes = req.getSchema().indexableUniqueKey(idStr);
     AtomicLong version = new AtomicLong();
-    SolrInputDocument doc = getInputDocument(req.getCore(), new BytesRef(idStr), version, null, Resolution.DOC);
+    SolrInputDocument doc = getInputDocument(req.getCore(), idBytes, idBytes, version, null, Resolution.ROOT_WITH_CHILDREN);
     log.info("getInputDocument called for id={}, returning {}", idStr, doc);
     rb.rsp.add("inputDocument", doc);
     rb.rsp.add("version", version.get());
@@ -397,36 +413,45 @@ public class RealTimeGetComponent extends SearchComponent
     }
   }
 
-  /***
+  /**
    * Given a partial document obtained from the transaction log (e.g. as a result of RTG), resolve to a full document
    * by populating all the partial updates that were applied on top of that last full document update.
-   * 
-   * @param onlyTheseFields When a non-null set of field names is passed in, the resolve process only attempts to populate
-   *        the given fields in this set. When this set is null, it resolves all fields.
+   * Transformers are applied.
+   * <p>TODO <em>Sometimes</em> there's copy-field target removal; it ought to be consistent.
+   *
+   * @param idBytes doc ID to find; never a child doc.
+   * @param partialDoc partial doc (an in-place update).  Could be a child doc, thus not having idBytes.
    * @return Returns the merged document, i.e. the resolved full document, or null if the document was not found (deleted
-   *          after the resolving began)
+   *          after the resolving began).  Never a child doc, since idBytes is never a child doc either.
    */
   private static SolrDocument resolveFullDocument(SolrCore core, BytesRef idBytes,
                                                   ReturnFields returnFields, SolrInputDocument partialDoc,
-                                                  @SuppressWarnings({"rawtypes"}) List logEntry,
-                                                  Set<String> onlyTheseFields) throws IOException {
+                                                  @SuppressWarnings({"rawtypes"}) List logEntry) throws IOException {
+    Set<String> onlyTheseFields = returnFields.getExplicitlyRequestedFieldNames();
     if (idBytes == null || (logEntry.size() != 5 && logEntry.size() != 6)) {
       throw new SolrException(ErrorCode.INVALID_STATE, "Either Id field not present in partial document or log entry doesn't have previous version.");
     }
     long prevPointer = (long) logEntry.get(UpdateLog.PREV_POINTER_IDX);
     long prevVersion = (long) logEntry.get(UpdateLog.PREV_VERSION_IDX);
+    final IndexSchema schema = core.getLatestSchema();
 
     // get the last full document from ulog
-    UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
-    long lastPrevPointer = ulog.applyPartialUpdates(idBytes, prevPointer, prevVersion, onlyTheseFields, partialDoc);
+    long lastPrevPointer;
+    // If partialDoc is NOT a child doc, then proceed and look into the ulog...
+    if (schema.printableUniqueKey(idBytes).equals(schema.printableUniqueKey(partialDoc))) {
+      UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
+      lastPrevPointer = ulog.applyPartialUpdates(idBytes, prevPointer, prevVersion, onlyTheseFields, partialDoc);
+    } else { // child doc.
+      // TODO could make this smarter but it's complicated with nested docs
+      lastPrevPointer = Long.MAX_VALUE; // results in reopenRealtimeSearcherAndGet
+    }
 
     if (lastPrevPointer == -1) { // full document was not found in tlog, but exists in index
-      SolrDocument mergedDoc = mergePartialDocWithFullDocFromIndex(core, idBytes, returnFields, onlyTheseFields, partialDoc);
-      return mergedDoc;
+      return mergePartialDocWithFullDocFromIndex(core, idBytes, returnFields, partialDoc);
     } else if (lastPrevPointer > 0) {
       // We were supposed to have found the last full doc also in the tlogs, but the prevPointer links led to nowhere
       // We should reopen a new RT searcher and get the doc. This should be a rare occurrence
-      Term idTerm = new Term(core.getLatestSchema().getUniqueKeyField().getName(), idBytes);
+      Term idTerm = new Term(schema.getUniqueKeyField().getName(), idBytes);
       SolrDocument mergedDoc = reopenRealtimeSearcherAndGet(core, idTerm, returnFields);
       if (mergedDoc == null) {
         return null; // the document may have been deleted as the resolving was going on.
@@ -438,12 +463,16 @@ public class RealTimeGetComponent extends SearchComponent
 
       // determine whether we can use the in place document, if the caller specified onlyTheseFields
       // and those fields are all supported for in-place updates
-      IndexSchema schema = core.getLatestSchema();
       boolean forInPlaceUpdate = onlyTheseFields != null
           && onlyTheseFields.stream().map(schema::getField)
           .allMatch(f -> null!=f && AtomicUpdateDocumentMerger.isSupportedFieldForInPlaceUpdate(f));
 
-      return toSolrDoc(partialDoc, schema, forInPlaceUpdate);
+      SolrDocument solrDoc = toSolrDoc(partialDoc, schema, forInPlaceUpdate); // filters copy-field targets TODO don't
+      DocTransformer transformer = returnFields.getTransformer();
+      if (transformer != null && !transformer.needsSolrIndexSearcher()) {
+        transformer.transform(solrDoc, -1); // no docId when from the ulog
+      } // if needs searcher, it must be [child]; tlog docs already have children
+      return solrDoc;
     }
   }
 
@@ -462,12 +491,7 @@ public class RealTimeGetComponent extends SearchComponent
       if (docid < 0) {
         return null;
       }
-      Document luceneDocument = searcher.doc(docid, returnFields.getLuceneFieldNames());
-      SolrDocument doc = toSolrDoc(luceneDocument, core.getLatestSchema());
-      SolrDocumentFetcher docFetcher = searcher.getDocFetcher();
-      docFetcher.decorateDocValueFields(doc, docid, docFetcher.getNonStoredDVs(false));
-
-      return doc;
+      return fetchSolrDoc(searcher, docid, returnFields);
     } finally {
       searcherHolder.decref();
     }
@@ -481,16 +505,14 @@ public class RealTimeGetComponent extends SearchComponent
    * @param core           A SolrCore instance, useful for obtaining a realtimesearcher and the schema
    * @param idBytes        Binary representation of the value of the unique key field
    * @param returnFields   Return fields, as requested
-   * @param onlyTheseFields When a non-null set of field names is passed in, the merge process only attempts to merge
-   *        the given fields in this set. When this set is null, it merges all fields.
    * @param partialDoc     A partial document (containing an in-place update) used for merging against a full document
    *                       from index; this maybe be null.
-   * @return If partial document is null, this returns document from the index or null if not found. 
+   * @return If partial document is null, this returns document from the index or null if not found.
    *         If partial document is not null, this returns a document from index merged with the partial document, or null if
    *         document doesn't exist in the index.
    */
   private static SolrDocument mergePartialDocWithFullDocFromIndex(SolrCore core, BytesRef idBytes, ReturnFields returnFields,
-             Set<String> onlyTheseFields, SolrInputDocument partialDoc) throws IOException {
+                                                                  SolrInputDocument partialDoc) throws IOException {
     RefCounted<SolrIndexSearcher> searcherHolder = core.getRealtimeSearcher(); //Searcher();
     try {
       // now fetch last document from index, and merge partialDoc on top of it
@@ -512,11 +534,10 @@ public class RealTimeGetComponent extends SearchComponent
         return doc;
       }
 
-      SolrDocument doc;
-      Set<String> decorateFields = onlyTheseFields == null ? searcher.getDocFetcher().getNonStoredDVs(false): onlyTheseFields;
-      Document luceneDocument = searcher.doc(docid, returnFields.getLuceneFieldNames());
-      doc = toSolrDoc(luceneDocument, core.getLatestSchema());
-      searcher.getDocFetcher().decorateDocValueFields(doc, docid, decorateFields);
+      SolrDocument doc = fetchSolrDoc(searcher, docid, returnFields);
+      if (!doc.containsKey(VERSION_FIELD)) {
+        searcher.getDocFetcher().decorateDocValueFields(doc, docid, Collections.singleton(VERSION_FIELD));
+      }
 
       long docVersion = (long) doc.getFirstValue(VERSION_FIELD);
       Object partialVersionObj = partialDoc.getFieldValue(VERSION_FIELD);
@@ -537,20 +558,88 @@ public class RealTimeGetComponent extends SearchComponent
     }
   }
 
+  /**
+   * Fetch the doc by the ID, returning the requested fields.
+   */
+  private static SolrDocument fetchSolrDoc(SolrIndexSearcher searcher, int docId, ReturnFields returnFields) throws IOException {
+    final SolrDocumentFetcher docFetcher = searcher.getDocFetcher();
+    final SolrDocument solrDoc = docFetcher.solrDoc(docId, (SolrReturnFields) returnFields);
+    final DocTransformer transformer = returnFields.getTransformer();
+    if (transformer != null) {
+      transformer.setContext(new RTGResultContext(returnFields, searcher, null)); // we get away with null req
+      transformer.transform(solrDoc, docId);
+    }
+    return solrDoc;
+  }
+
+  private static void removeCopyFieldTargets(SolrDocument solrDoc, IndexSchema schema) {
+    // TODO ideally we wouldn't have fetched these in the first place!
+    final Iterator<Map.Entry<String, Object>> iterator = solrDoc.iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<String, Object> fieldVal =  iterator.next();
+      String fieldName = fieldVal.getKey();
+      SchemaField sf = schema.getFieldOrNull(fieldName);
+      if (sf != null && schema.isCopyFieldTarget(sf)) {
+        iterator.remove();
+      }
+    }
+  }
+
   public static SolrInputDocument DELETED = new SolrInputDocument();
 
+  @Deprecated // need Resolution
+  public static SolrInputDocument getInputDocumentFromTlog(SolrCore core, BytesRef idBytes, AtomicLong versionReturned,
+                                                           Set<String> onlyTheseNonStoredDVs, boolean resolveFullDocument) {
+    return getInputDocumentFromTlog(core, idBytes, versionReturned, onlyTheseNonStoredDVs,
+        resolveFullDocument ? Resolution.DOC : Resolution.PARTIAL);
+  }
+
+  /**
+   * Specialized to pick out a child doc from a nested doc from the TLog.
+   * @see #getInputDocumentFromTlog(SolrCore, BytesRef, AtomicLong, Set, Resolution)
+   */
+  private static SolrInputDocument getInputDocumentFromTlog(
+      SolrCore core,
+      BytesRef idBytes,
+      BytesRef rootIdBytes,
+      AtomicLong versionReturned,
+      Set<String> onlyTheseFields,
+      Resolution resolution) {
+    if (idBytes.equals(rootIdBytes)) { // simple case; not looking for a child
+      return getInputDocumentFromTlog(
+          core, rootIdBytes, versionReturned, onlyTheseFields, resolution);
+    }
+
+    // Ensure we request the ID to pick out the child doc in the nest
+    final String uniqueKeyField = core.getLatestSchema().getUniqueKeyField().getName();
+    if (onlyTheseFields != null && !onlyTheseFields.contains(uniqueKeyField)) {
+      onlyTheseFields = new HashSet<>(onlyTheseFields); // clone
+      onlyTheseFields.add(uniqueKeyField);
+    }
+
+    SolrInputDocument iDoc =
+        getInputDocumentFromTlog(
+            core, rootIdBytes, versionReturned, onlyTheseFields, Resolution.ROOT_WITH_CHILDREN);
+    if (iDoc == DELETED || iDoc == null) {
+      return iDoc;
+    }
+
+    iDoc = findNestedDocById(iDoc, idBytes, core.getLatestSchema());
+    if (iDoc == null) {
+      return DELETED; // new nest overwrote the old nest without the ID we are looking for?
+    }
+    return iDoc;
+  }
+
   /** returns the SolrInputDocument from the current tlog, or DELETED if it has been deleted, or
    * null if there is no record of it in the current update log.  If null is returned, it could
-   * still be in the latest index.
+   * still be in the latest index.  Copy-field target fields are excluded.
+   * @param idBytes doc ID to find; never a child doc.
    * @param versionReturned If a non-null AtomicLong is passed in, it is set to the version of the update returned from the TLog.
-   * @param resolveFullDocument In case the document is fetched from the tlog, it could only be a partial document if the last update
-   *                  was an in-place update. In that case, should this partial document be resolved to a full document (by following
-   *                  back prevPointer/prevVersion)?
    */
   @SuppressWarnings({"fallthrough"})
   public static SolrInputDocument getInputDocumentFromTlog(SolrCore core, BytesRef idBytes, AtomicLong versionReturned,
-      Set<String> onlyTheseNonStoredDVs, boolean resolveFullDocument) {
-
+      Set<String> onlyTheseFields, Resolution resolution) {
     UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
 
     if (ulog != null) {
@@ -568,17 +657,17 @@ public class RealTimeGetComponent extends SearchComponent
           case UpdateLog.UPDATE_INPLACE:
             assert entry.size() == 5;
             
-            if (resolveFullDocument) {
+            if (resolution != Resolution.PARTIAL) {
               SolrInputDocument doc = (SolrInputDocument)entry.get(entry.size()-1);
               try {
                 // For in-place update case, we have obtained the partial document till now. We need to
                 // resolve it to a full document to be returned to the user.
-                SolrDocument sdoc = resolveFullDocument(core, idBytes, new SolrReturnFields(), doc, entry, onlyTheseNonStoredDVs);
+                SolrReturnFields returnFields = makeReturnFields(core, onlyTheseFields, resolution);
+                SolrDocument sdoc = resolveFullDocument(core, idBytes, returnFields, doc, entry);
                 if (sdoc == null) {
                   return DELETED;
                 }
-                doc = toSolrInputDocument(sdoc, core.getLatestSchema());
-                return doc;
+                return toSolrInputDocument(sdoc, core.getLatestSchema()); // filters copy-field
               } catch (IOException ex) {
                 throw new SolrException(ErrorCode.SERVER_ERROR, "Error while resolving full document. ", ex);
               }
@@ -598,87 +687,64 @@ public class RealTimeGetComponent extends SearchComponent
     return null;
   }
 
-  /**
-   * Obtains the latest document for a given id from the tlog or index (if not found in the tlog).
-   * 
-   * NOTE: This method uses the effective value for nonStoredDVs as null in the call to @see {@link RealTimeGetComponent#getInputDocument(SolrCore, BytesRef, AtomicLong, Set, Resolution)},
-   * so as to retrieve all stored and non-stored DV fields from all documents.
-   */
-
+  @Deprecated // easy to use wrong
   public static SolrInputDocument getInputDocument(SolrCore core, BytesRef idBytes, Resolution lookupStrategy) throws IOException {
-    return getInputDocument (core, idBytes, null, null, lookupStrategy);
+    return getInputDocument (core, idBytes, idBytes, null, null, lookupStrategy);
   }
-  
+
   /**
-   * Obtains the latest document for a given id from the tlog or through the realtime searcher (if not found in the tlog). 
+   * Obtains the latest document for a given id from the tlog or through the realtime searcher (if not found in the tlog).
+   * Fields that are targets of copy-fields are excluded.
+   *
+   * @param idBytes ID of the document to be fetched.
+   * @param rootIdBytes the root ID of the document being looked up.
+   *                    If there are no child docs, this is always the same as idBytes.
    * @param versionReturned If a non-null AtomicLong is passed in, it is set to the version of the update returned from the TLog.
-   * @param onlyTheseNonStoredDVs If not-null, populate only these DV fields in the document fetched through the realtime searcher. 
-   *                  If this is null, decorate all non-stored  DVs (that are not targets of copy fields) from the searcher.
-   *                  When non-null, stored fields are not fetched.
-   * @param resolveStrategy The strategy to resolve the the document.
+   * @param onlyTheseFields If not-null, this limits the fields that are returned.  However it is only an optimization
+   *                        hint since other fields may be returned.  Copy field targets are never returned.
+   * @param resolveStrategy {@link Resolution#DOC} or {@link Resolution#ROOT_WITH_CHILDREN}.
    * @see Resolution
    */
-  public static SolrInputDocument getInputDocument(SolrCore core, BytesRef idBytes, AtomicLong versionReturned,
-      Set<String> onlyTheseNonStoredDVs, Resolution resolveStrategy) throws IOException {
-    SolrInputDocument sid = null;
-    RefCounted<SolrIndexSearcher> searcherHolder = null;
-    try {
-      SolrIndexSearcher searcher = null;
-      sid = getInputDocumentFromTlog(core, idBytes, versionReturned, onlyTheseNonStoredDVs, true);
-      if (sid == DELETED) {
-        return null;
-      }
-
-      if (sid == null) {
-        // didn't find it in the update log, so it should be in the newest searcher opened
-        if (searcher == null) {
-          searcherHolder = core.getRealtimeSearcher();
-          searcher = searcherHolder.get();
-        }
-
-        // SolrCore.verbose("RealTimeGet using searcher ", searcher);
-        final IndexSchema schema = core.getLatestSchema();
-        SchemaField idField = schema.getUniqueKeyField();
-
-        int docid = searcher.getFirstMatch(new Term(idField.getName(), idBytes));
-        if (docid < 0) return null;
+  public static SolrInputDocument getInputDocument(SolrCore core, BytesRef idBytes, BytesRef rootIdBytes, AtomicLong versionReturned,
+                                                   Set<String> onlyTheseFields, Resolution resolveStrategy) throws IOException {
+    assert resolveStrategy != Resolution.PARTIAL;
+    assert resolveStrategy == Resolution.DOC || idBytes.equals(rootIdBytes); // not needed (yet)
+
+    SolrInputDocument sid =
+        getInputDocumentFromTlog(
+            core, idBytes, rootIdBytes, versionReturned, onlyTheseFields, resolveStrategy);
+    if (sid == DELETED) {
+      return null;
+    }
 
-        SolrDocumentFetcher docFetcher = searcher.getDocFetcher();
-        if (onlyTheseNonStoredDVs != null) {
-          sid = new SolrInputDocument();
-        } else {
-          Document luceneDocument = docFetcher.doc(docid);
-          sid = toSolrInputDocument(luceneDocument, schema);
-        }
-        final boolean isNestedRequest = resolveStrategy == Resolution.DOC_WITH_CHILDREN || resolveStrategy == Resolution.ROOT_WITH_CHILDREN;
-        decorateDocValueFields(docFetcher, sid, docid, onlyTheseNonStoredDVs, isNestedRequest || schema.hasExplicitField(IndexSchema.NEST_PATH_FIELD_NAME));
-        SolrInputField rootField = sid.getField(IndexSchema.ROOT_FIELD_NAME);
-        if((isNestedRequest) && schema.isUsableForChildDocs() && schema.hasExplicitField(IndexSchema.NEST_PATH_FIELD_NAME) && rootField!=null) {
-          // doc is part of a nested structure
-          final boolean resolveRootDoc = resolveStrategy == Resolution.ROOT_WITH_CHILDREN;
-          String id = resolveRootDoc? (String) rootField.getFirstValue(): (String) sid.getField(idField.getName()).getFirstValue();
-          ModifiableSolrParams params = new ModifiableSolrParams()
-              .set("fl", "*, _nest_path_, [child]")
-              .set("limit", "-1");
-          SolrQueryRequest nestedReq = new LocalSolrQueryRequest(core, params);
-          final BytesRef rootIdBytes = new BytesRef(id);
-          final int rootDocId = searcher.getFirstMatch(new Term(idField.getName(), rootIdBytes));
-          final DocTransformer childDocTransformer = core.getTransformerFactory("child").create("child", params, nestedReq);
-          final ResultContext resultContext = new RTGResultContext(new SolrReturnFields(nestedReq), searcher, nestedReq);
-          childDocTransformer.setContext(resultContext);
-          final SolrDocument nestedDoc;
-          if(resolveRootDoc && rootIdBytes.equals(idBytes)) {
-            nestedDoc = toSolrDoc(sid, schema);
-          } else {
-            nestedDoc = toSolrDoc(docFetcher.doc(rootDocId), schema);
-            decorateDocValueFields(docFetcher, nestedDoc, rootDocId, onlyTheseNonStoredDVs, true);
+    if (sid == null) {
+      // didn't find it in the update log, so it should be in the newest searcher opened
+      RefCounted<SolrIndexSearcher> searcherHolder = core.getRealtimeSearcher();
+      try {
+        SolrIndexSearcher searcher = searcherHolder.get();
+
+        int docId =
+            searcher.getFirstMatch(
+                new Term(
+                    core.getLatestSchema().getUniqueKeyField().getName(),
+                    resolveStrategy == Resolution.ROOT_WITH_CHILDREN ? rootIdBytes : idBytes));
+        if (docId < 0) return null;
+
+        if (resolveStrategy == Resolution.ROOT_WITH_CHILDREN
+            && core.getLatestSchema().isUsableForChildDocs()) {
+          // check that this doc is in fact a root document as a prevention measure
+          if (!hasRootTerm(searcher, rootIdBytes)) {
+            throw new SolrException(
+                ErrorCode.BAD_REQUEST,
+                "Attempted an atomic/partial update to a child doc without indicating the _root_ somehow.");
           }
-          childDocTransformer.transform(nestedDoc, rootDocId);
-          sid = toSolrInputDocument(nestedDoc, schema);
         }
-      }
-    } finally {
-      if (searcherHolder != null) {
+
+        SolrDocument solrDoc =
+            fetchSolrDoc(searcher, docId, makeReturnFields(core, onlyTheseFields, resolveStrategy));
+        sid = toSolrInputDocument(solrDoc, core.getLatestSchema()); // filters copy-field targets
+        // the assertions above furthermore guarantee the result corresponds to idBytes
+      } finally {
         searcherHolder.decref();
       }
     }
@@ -691,38 +757,51 @@ public class RealTimeGetComponent extends SearchComponent
     return sid;
   }
 
-  private static void decorateDocValueFields(SolrDocumentFetcher docFetcher,
-                                             @SuppressWarnings({"rawtypes"})SolrDocumentBase doc, int docid, Set<String> onlyTheseNonStoredDVs, boolean resolveNestedFields) throws IOException {
-    if (onlyTheseNonStoredDVs != null) {
-      docFetcher.decorateDocValueFields(doc, docid, onlyTheseNonStoredDVs);
-    } else {
-      docFetcher.decorateDocValueFields(doc, docid, docFetcher.getNonStoredDVsWithoutCopyTargets());
-    }
-    if(resolveNestedFields) {
-      docFetcher.decorateDocValueFields(doc, docid, NESTED_META_FIELDS);
+  private static boolean hasRootTerm(SolrIndexSearcher searcher, BytesRef rootIdBytes) throws IOException {
+    final String fieldName = IndexSchema.ROOT_FIELD_NAME;
+    final List<LeafReaderContext> leafContexts = searcher.getTopReaderContext().leaves();
+    for (final LeafReaderContext leaf : leafContexts) {
+      final LeafReader reader = leaf.reader();
+
+      final Terms terms = reader.terms(fieldName);
+      if (terms == null) continue;
+
+      TermsEnum te = terms.iterator();
+      if (te.seekExact(rootIdBytes)) {
+        return true;
+      }
     }
+    return false;
   }
 
-  private static SolrInputDocument toSolrInputDocument(Document doc, IndexSchema schema) {
-    SolrInputDocument out = new SolrInputDocument();
-    for( IndexableField f : doc.getFields() ) {
-      String fname = f.name();
-      SchemaField sf = schema.getFieldOrNull(f.name());
-      Object val = null;
-      if (sf != null) {
-        if ((!sf.hasDocValues() && !sf.stored()) || schema.isCopyFieldTarget(sf)) continue;
-        val = sf.getType().toObject(f);   // object or external string?
-      } else {
-        val = f.stringValue();
-        if (val == null) val = f.numericValue();
-        if (val == null) val = f.binaryValue();
-        if (val == null) val = f;
+  /** Traverse the doc looking for a doc with the specified ID. */
+  private static SolrInputDocument findNestedDocById(SolrInputDocument iDoc, BytesRef idBytes, IndexSchema schema) {
+    assert schema.printableUniqueKey(iDoc) != null : "need IDs";
+    // traverse nested doc, looking for the node with the ID we are looking for
+    SolrInputDocument[] found = new SolrInputDocument[1];
+    String idStr = schema.printableUniqueKey(idBytes);
+    BiConsumer<String, SolrInputDocument> finder = (label, childDoc) -> {
+      if (found[0] == null && idStr.equals(schema.printableUniqueKey(childDoc))) {
+        found[0] = childDoc;
       }
+    };
+    iDoc.visitSelfAndNestedDocs(finder);
+    return found[0];
+  }
 
-      // todo: how to handle targets of copy fields (including polyfield sub-fields)?
-      out.addField(fname, val);
+  private static SolrReturnFields makeReturnFields(SolrCore core, Set<String> requestedFields, Resolution resolution) {
+    DocTransformer docTransformer;
+    if (resolution == Resolution.ROOT_WITH_CHILDREN && core.getLatestSchema().isUsableForChildDocs()) {
+      SolrParams params = new ModifiableSolrParams().set("limit", "-1");
+      try (LocalSolrQueryRequest req = new LocalSolrQueryRequest(core, params)) {
+        docTransformer = core.getTransformerFactory("child").create(null, params, req);
+      }
+    } else {
+      docTransformer = null;
     }
-    return out;
+    // TODO optimization: add feature to SolrReturnFields to exclude copyFieldTargets from wildcard matching.
+    //   Today, we filter this data out later before returning, but it's already been fetched.
+    return new SolrReturnFields(requestedFields, docTransformer);
   }
 
   private static SolrInputDocument toSolrInputDocument(SolrDocument doc, IndexSchema schema) {
@@ -835,7 +914,6 @@ public class RealTimeGetComponent extends SearchComponent
    *                         see {@link DocumentBuilder#toDocument(SolrInputDocument, IndexSchema, boolean, boolean)}
    */
   public static SolrDocument toSolrDoc(SolrInputDocument sdoc, IndexSchema schema, boolean forInPlaceUpdate) {
-    // TODO what about child / nested docs?
     // TODO: do something more performant than this double conversion
     Document doc = DocumentBuilder.toDocument(sdoc, schema, forInPlaceUpdate, true);
 
@@ -1251,27 +1329,16 @@ public class RealTimeGetComponent extends SearchComponent
   }
 
   /**
-   *  <p>
-   *    Lookup strategy for {@link #getInputDocument(SolrCore, BytesRef, AtomicLong, Set, Resolution)}.
-   *  </p>
-   *  <ul>
-   *    <li>{@link #DOC}</li>
-   *    <li>{@link #DOC_WITH_CHILDREN}</li>
-   *    <li>{@link #ROOT_WITH_CHILDREN}</li>
-   *  </ul>
+   * Lookup strategy for some methods on this class.
    */
   public static enum Resolution {
-    /**
-     * Resolve this partial document to a full document (by following back prevPointer/prevVersion)?
-     */
+    /** A partial update document.  Whole documents may still be returned. */
+    PARTIAL,
+
+    /** Resolve to a whole document, exclusive of children. */
     DOC,
-    /**
-     * Check whether the document has child documents. If so, return the document including its children.
-     */
-    DOC_WITH_CHILDREN,
-    /**
-     * Check whether the document is part of a nested hierarchy. If so, return the whole hierarchy(look up root doc).
-     */
+
+    /** Resolves the whole nested hierarchy (look up root doc). */
     ROOT_WITH_CHILDREN
   }
 
diff --git a/solr/core/src/java/org/apache/solr/schema/IndexSchema.java b/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
index d41f395..41b6d8a 100644
--- a/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
+++ b/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
@@ -54,6 +54,7 @@ import org.apache.solr.common.MapSerializable;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.cloud.SolrClassLoader;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.MapSolrParams;
@@ -360,6 +361,21 @@ public class IndexSchema {
     }
   }
 
+  /** Like {@link #printableUniqueKey(org.apache.lucene.document.Document)} */
+  public String printableUniqueKey(SolrInputDocument solrDoc) {
+    Object val = solrDoc.getFieldValue(uniqueKeyFieldName);
+    if (val == null) {
+      return null;
+    } else {
+      return val.toString();
+    }
+  }
+
+  /** Given an indexable uniqueKey value, return the readable/printable version */
+  public String printableUniqueKey(BytesRef idBytes) {
+    return uniqueKeyFieldType.indexedToReadable(idBytes.utf8ToString());
+  }
+
   /** Given a readable/printable uniqueKey value, return an indexable version */
   public BytesRef indexableUniqueKey(String idStr) {
     return new BytesRef(uniqueKeyFieldType.toInternal(idStr));
@@ -1936,32 +1952,6 @@ public class IndexSchema {
             rootType.getTypeName().equals(uniqueKeyFieldType.getTypeName()));
   }
 
-  /**
-   * Helper method that returns <code>true</code> if the {@link #ROOT_FIELD_NAME} uses the exact
-   * same 'type' as the {@link #getUniqueKeyField()} and has {@link #NEST_PATH_FIELD_NAME}
-   * defined as a {@link NestPathField}
-   * @lucene.internal
-   */
-  public boolean savesChildDocRelations() {
-    //TODO make this boolean a field so it needn't be looked up each time?
-    if (!isUsableForChildDocs()) {
-      return false;
-    }
-    FieldType nestPathType = getFieldTypeNoEx(NEST_PATH_FIELD_NAME);
-    return nestPathType instanceof NestPathField;
-  }
-
-  /**
-   * Does this schema supports partial updates (aka atomic updates) and child docs as well.
-   */
-  public boolean supportsPartialUpdatesOfChildDocs() {
-    if (savesChildDocRelations() == false) {
-      return false;
-    }
-    SchemaField rootField = getField(IndexSchema.ROOT_FIELD_NAME);
-    return rootField.stored() || rootField.hasDocValues();
-  }
-
   public PayloadDecoder getPayloadDecoder(String field) {
     FieldType ft = getFieldType(field);
     if (ft == null)
diff --git a/solr/core/src/java/org/apache/solr/schema/NestPathField.java b/solr/core/src/java/org/apache/solr/schema/NestPathField.java
index 926aa7e..eb3de89 100644
--- a/solr/core/src/java/org/apache/solr/schema/NestPathField.java
+++ b/solr/core/src/java/org/apache/solr/schema/NestPathField.java
@@ -40,6 +40,7 @@ public class NestPathField extends SortableTextField {
   @Override
   public void setArgs(IndexSchema schema, Map<String, String> args) {
     args.putIfAbsent("stored", "false");
+    args.putIfAbsent("multiValued", "false");
     args.putIfAbsent("omitTermFreqAndPositions", "true");
     args.putIfAbsent("omitNorms", "true");
     args.putIfAbsent("maxCharsForDocValues", "-1");
diff --git a/solr/core/src/java/org/apache/solr/search/SolrReturnFields.java b/solr/core/src/java/org/apache/solr/search/SolrReturnFields.java
index 9a04c25..6135bb2 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrReturnFields.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrReturnFields.java
@@ -17,6 +17,7 @@
 package org.apache.solr.search;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.LinkedHashSet;
@@ -123,6 +124,34 @@ public class SolrReturnFields extends ReturnFields {
     parseFieldList(fl, req);
   }
 
+  /**
+   * For pre-parsed simple field list with optional transformer.
+   * Does not support globs or the score.
+   * This constructor is more for internal use; not for parsing user input.
+   *
+   * @param plainFields simple field list; nothing special. If null, equivalent to all-fields.
+   * @param docTransformer optional transformer.
+   */
+  public SolrReturnFields(Collection<String> plainFields, DocTransformer docTransformer) {
+    if (plainFields != null) {
+      _wantsAllFields = false;
+      for (String field : plainFields) {
+        assert field.indexOf('*') == -1 && !field.equals(SCORE);
+        addField(field, null, null, false);
+      }
+    } else {
+      _wantsAllFields = true;
+    }
+    if (docTransformer != null) {
+      transformer = docTransformer;
+      // doc transformer can request extra fields.
+      String[] extraRequestFields = docTransformer.getExtraRequestFields();
+      if (extraRequestFields != null) {
+        Collections.addAll(fields, extraRequestFields); // do NOT call addField
+      }
+    }
+  }
+
   public RetrieveFieldsOptimizer getFetchOptimizer(Supplier<RetrieveFieldsOptimizer> supplier) {
     if (fetchOptimizer == null) {
       fetchOptimizer = supplier.get();
diff --git a/solr/core/src/java/org/apache/solr/update/AddUpdateCommand.java b/solr/core/src/java/org/apache/solr/update/AddUpdateCommand.java
index 1e4384d..2de9e6b 100644
--- a/solr/core/src/java/org/apache/solr/update/AddUpdateCommand.java
+++ b/solr/core/src/java/org/apache/solr/update/AddUpdateCommand.java
@@ -18,15 +18,18 @@ package org.apache.solr.update;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.SolrInputField;
+import org.apache.solr.common.cloud.DocRouter;
+import org.apache.solr.common.cloud.ImplicitDocRouter;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ShardParams;
 import org.apache.solr.request.SolrQueryRequest;
@@ -39,6 +42,9 @@ import org.apache.solr.schema.SchemaField;
  */
 public class AddUpdateCommand extends UpdateCommand {
 
+  /** In some limited circumstances of child docs, this holds the _route_ param. */
+  final String useRouteAsRoot; // lets hope this goes away in SOLR-15064
+
   /**
    * Higher level SolrInputDocument, normally used to construct the Lucene Document(s)
    * to index.
@@ -64,15 +70,35 @@ public class AddUpdateCommand extends UpdateCommand {
 
   public boolean isLastDocInBatch = false;
 
-  /** Is this a nested update, null means not yet calculated. */
-  public Boolean isNested = null;
-
   // optional id in "internal" indexed form... if it is needed and not supplied,
   // it will be obtained from the doc.
   private BytesRef indexedId;
+  private String indexedIdStr;
+  private String childDocIdStr;
 
   public AddUpdateCommand(SolrQueryRequest req) {
     super(req);
+
+    // Populate useRouteParamAsIndexedId.
+    // This ought to be deprecated functionality that we remove in 9.0. SOLR-15064
+    String route = null;
+    if (req != null) { // some tests use no req
+      route = req.getParams().get(ShardParams._ROUTE_);
+      if (route == null || !req.getSchema().isUsableForChildDocs()) {
+        route = null;
+      } else {
+        // use route but there's one last exclusion: It's incompatible with SolrCloud implicit router.
+        String collectionName = req.getCore().getCoreDescriptor().getCollectionName();
+        if (collectionName != null) {
+          DocRouter router = req.getCore().getCoreContainer().getZkController().getClusterState()
+              .getCollection(collectionName).getRouter();
+          if (router instanceof ImplicitDocRouter) {
+            route = null;
+          }
+        }
+      }
+    }
+    useRouteAsRoot = route;
   }
 
   @Override
@@ -84,6 +110,8 @@ public class AddUpdateCommand extends UpdateCommand {
    public void clear() {
      solrDoc = null;
      indexedId = null;
+     indexedIdStr = null;
+     childDocIdStr = null;
      updateTerm = null;
      isLastDocInBatch = false;
      version = 0;
@@ -95,121 +123,130 @@ public class AddUpdateCommand extends UpdateCommand {
    }
 
   /**
-   * Creates and returns a lucene Document to index.
-   * Nested documents, if found, will cause an exception to be thrown.  Call {@link #getLuceneDocsIfNested()} for that.
+   * Creates and returns a lucene Document for in-place update.
+   * The SolrInputDocument itself may be modified, which will be reflected in the update log.
    * Any changes made to the returned Document will not be reflected in the SolrInputDocument, or future calls to this
    * method.
-   * Note that the behavior of this is sensitive to {@link #isInPlaceUpdate()}.*/
-   public Document getLuceneDocument() {
-     final boolean ignoreNestedDocs = false; // throw an exception if found
-     SolrInputDocument solrInputDocument = getSolrInputDocument();
-     if (!isInPlaceUpdate() && getReq().getSchema().isUsableForChildDocs()) {
-       addRootField(solrInputDocument, getRootIdUsingRouteParam());
-     }
-     return DocumentBuilder.toDocument(solrInputDocument, req.getSchema(), isInPlaceUpdate(), ignoreNestedDocs);
-   }
-
-  /** Returns the indexed ID for this document.  The returned BytesRef is retained across multiple calls, and should not be modified. */
-   public BytesRef getIndexedId() {
-     if (indexedId == null) {
-       IndexSchema schema = req.getSchema();
-       SchemaField sf = schema.getUniqueKeyField();
-       if (sf != null) {
-         if (solrDoc != null) {
-           SolrInputField field = solrDoc.getField(sf.getName());
-
-           int count = field==null ? 0 : field.getValueCount();
-           if (count == 0) {
-             if (overwrite) {
-               throw new SolrException( SolrException.ErrorCode.BAD_REQUEST,"Document is missing mandatory uniqueKey field: " + sf.getName());
-             }
-           } else if (count  > 1) {
-             throw new SolrException( SolrException.ErrorCode.BAD_REQUEST,"Document contains multiple values for uniqueKey field: " + field);
-           } else {
-             BytesRefBuilder b = new BytesRefBuilder();
-             sf.getType().readableToIndexed(field.getFirstValue().toString(), b);
-             indexedId = b.get();
-           }
-         }
-       }
+   */
+   Document makeLuceneDocForInPlaceUpdate() {
+     // perhaps this should move to UpdateHandler or DocumentBuilder?
+     assert isInPlaceUpdate();
+     if (req.getSchema().isUsableForChildDocs() && solrDoc.getField(IndexSchema.ROOT_FIELD_NAME) == null) {
+       solrDoc.setField(IndexSchema.ROOT_FIELD_NAME, getIndexedIdStr());
      }
-     return indexedId;
-   }
-
-   public void setIndexedId(BytesRef indexedId) {
-     this.indexedId = indexedId;
+     final boolean forInPlaceUpdate = true;
+     final boolean ignoreNestedDocs = false; // throw an exception if found
+     return DocumentBuilder.toDocument(solrDoc, req.getSchema(), forInPlaceUpdate, ignoreNestedDocs);
    }
 
-   public String getPrintableId() {
-    if (req != null) {
-      IndexSchema schema = req.getSchema();
-      SchemaField sf = schema.getUniqueKeyField();
-      if (solrDoc != null && sf != null) {
-        SolrInputField field = solrDoc.getField(sf.getName());
-        if (field != null) {
-          return field.getFirstValue().toString();
-        }
-      }
-    }
-     return "(null)";
-   }
+  /**
+   * Returns the indexed ID for this document, or the root ID for nested documents.
+   *
+   * @return possibly null if there's no uniqueKey field
+   */
+  public String getIndexedIdStr() {
+    extractIdsIfNeeded();
+    return indexedIdStr;
+  }
 
   /**
+   * Returns the indexed ID for this document, or the root ID for nested documents. The returned
+   * BytesRef should be treated as immutable. It will not be re-used/modified for additional docs.
    *
-   * @return value of _route_ param({@link ShardParams#_ROUTE_}), otherwise doc id.
+   * @return possibly null if there's no uniqueKey field
    */
-  public String getRootIdUsingRouteParam() {
-     return req.getParams().get(ShardParams._ROUTE_, getHashableId());
-   }
+  public BytesRef getIndexedId() {
+    extractIdsIfNeeded();
+    return indexedId;
+  }
 
   /**
-   * @return String id to hash
+   * Returns the ID of the doc itself, possibly different from {@link #getIndexedIdStr()} which
+   * points to the root doc.
+   *
+   * @return possibly null if there's no uniqueKey field
    */
-  public String getHashableId() {
+  public String getChildDocIdStr() {
+    extractIdsIfNeeded();
+    return childDocIdStr;
+  }
+
+  /** The ID for logging purposes. */
+  public String getPrintableId() {
+    if (req == null) {
+      return "(uninitialized)"; // in tests?
+    }
+    extractIdsIfNeeded();
+    if (indexedIdStr == null) {
+      return "(null)";
+    } else if (indexedIdStr.equals(childDocIdStr)) {
+      return indexedIdStr;
+    } else {
+      return childDocIdStr + " (root=" + indexedIdStr + ")";
+    }
+  }
+
+  private void extractIdsIfNeeded() {
+    if (indexedId != null) {
+      return;
+    }
     IndexSchema schema = req.getSchema();
     SchemaField sf = schema.getUniqueKeyField();
     if (sf != null) {
       if (solrDoc != null) {
         SolrInputField field = solrDoc.getField(sf.getName());
-
-        int count = field == null ? 0 : field.getValueCount();
+        // check some uniqueKey constraints
+        int count = field==null ? 0 : field.getValueCount();
         if (count == 0) {
           if (overwrite) {
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-                "Document is missing mandatory uniqueKey field: "
-                    + sf.getName());
+            throw new SolrException( SolrException.ErrorCode.BAD_REQUEST,"Document is missing mandatory uniqueKey field: " + sf.getName());
           }
-        } else if (count > 1) {
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-              "Document contains multiple values for uniqueKey field: " + field);
+        } else if (count  > 1) {
+          throw new SolrException( SolrException.ErrorCode.BAD_REQUEST,"Document contains multiple values for uniqueKey field: " + field);
         } else {
-          return field.getFirstValue().toString();
+          this.childDocIdStr = field.getFirstValue().toString();
+          // the root might be in _root_ field or _route_ param.  If neither, then uniqueKeyField.
+          this.indexedIdStr = (String) solrDoc.getFieldValue(IndexSchema.ROOT_FIELD_NAME); // or here
+          if (this.indexedIdStr == null) {
+            this.indexedIdStr = useRouteAsRoot;
+            if (this.indexedIdStr == null) {
+              this.indexedIdStr = childDocIdStr;
+            }
+          }
+          indexedId = schema.indexableUniqueKey(indexedIdStr);
         }
       }
     }
-    return null;
+  }
+
+  @VisibleForTesting
+  public void setIndexedId(BytesRef indexedId) {
+    this.indexedId = indexedId;
+    this.indexedIdStr = indexedId.utf8ToString();
+    this.childDocIdStr = indexedIdStr;
   }
 
   /**
-   * Computes the final flattened Solr docs that are ready to be converted to Lucene docs.  If no flattening is
-   * performed then we return null, and the caller ought to use {@link #getLuceneDocument()} instead.
+   * Computes the final flattened Lucene docs, possibly generating them on-demand (on iteration).
+   * The SolrInputDocument itself may be modified, which will be reflected in the update log.
    * This should only be called once.
    * Any changes made to the returned Document(s) will not be reflected in the SolrInputDocument,
    * or future calls to this method.
    */
-  public Iterable<Document> getLuceneDocsIfNested() {
+  Iterable<Document> makeLuceneDocs() {
+    // perhaps this should move to UpdateHandler or DocumentBuilder?
     assert ! isInPlaceUpdate() : "We don't expect this to happen."; // but should "work"?
     if (!req.getSchema().isUsableForChildDocs()) {
       // note if the doc is nested despite this, we'll throw an exception elsewhere
-      return null;
+      final boolean forInPlaceUpdate = false;
+      final boolean ignoreNestedDocs = false; // throw an exception if found
+      Document doc = DocumentBuilder.toDocument(solrDoc, req.getSchema(), forInPlaceUpdate, ignoreNestedDocs);
+      return Collections.singleton(doc);
     }
 
     List<SolrInputDocument> all = flatten(solrDoc);
-    if (all.size() <= 1) {
-      return null; // caller should call getLuceneDocument() instead
-    }
 
-    final String rootId = getRootIdUsingRouteParam();
+    final String rootId = getIndexedIdStr();
     final SolrInputField versionSif = solrDoc.get(CommonParams.VERSION_FIELD);
 
     for (SolrInputDocument sdoc : all) {
diff --git a/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java b/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
index 523a35d..37ef433 100644
--- a/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
+++ b/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
@@ -319,12 +319,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
     RefCounted<IndexWriter> iw = solrCoreState.getIndexWriter(core);
     try {
       IndexWriter writer = iw.get();
-      Iterable<Document> nestedDocs = cmd.getLuceneDocsIfNested();
-      if (nestedDocs != null) {
-        writer.addDocuments(nestedDocs);
-      } else {
-        writer.addDocument(cmd.getLuceneDocument());
-      }
+      writer.addDocuments(cmd.makeLuceneDocs());
       if (ulog != null) ulog.add(cmd);
 
     } finally {
@@ -425,7 +420,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
       return;
     }
 
-    Term deleteTerm = getIdTerm(cmd.getIndexedId(), false);
+    Term deleteTerm = getIdTerm(cmd.getIndexedId());
     // SolrCore.verbose("deleteDocuments",deleteTerm,writer);
     RefCounted<IndexWriter> iw = solrCoreState.getIndexWriter(core);
     try {
@@ -932,7 +927,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
    * needed based on {@link AddUpdateCommand#isInPlaceUpdate}.
    * <p>
    * If the this is an UPDATE_INPLACE cmd, then all fields included in 
-   * {@link AddUpdateCommand#getLuceneDocument} must either be the uniqueKey field, or be DocValue 
+   * {@link AddUpdateCommand#makeLuceneDocForInPlaceUpdate} must either be the uniqueKey field, or be DocValue
    * only fields.
    * </p>
    *
@@ -949,33 +944,21 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
       }
       // we don't support the solrInputDoc with nested child docs either but we'll throw an exception if attempted
 
-      Term updateTerm = new Term(idField.getName(), cmd.getIndexedId());
-      Document luceneDocument = cmd.getLuceneDocument();
-
-      final List<IndexableField> origDocFields = luceneDocument.getFields();
-      final List<Field> fieldsToUpdate = new ArrayList<>(origDocFields.size());
-      for (IndexableField field : origDocFields) {
-        if (! field.name().equals(updateTerm.field()) ) {
-          fieldsToUpdate.add((Field)field);
-        }
-      }
+      // can't use cmd.getIndexedId because it will be a root doc if this doc is a child
+      Term updateTerm = new Term(idField.getName(),
+          core.getLatestSchema().indexableUniqueKey(cmd.getChildDocIdStr()));
+      List<IndexableField> fields = cmd.makeLuceneDocForInPlaceUpdate().getFields(); // skips uniqueKey and _root_
       log.debug("updateDocValues({})", cmd);
-      writer.updateDocValues(updateTerm, fieldsToUpdate.toArray(new Field[fieldsToUpdate.size()]));
+      writer.updateDocValues(updateTerm, fields.toArray(new Field[fields.size()]));
 
     } else { // more normal path
 
-      Iterable<Document> nestedDocs = cmd.getLuceneDocsIfNested();
-      boolean isNested = nestedDocs != null; // AKA nested child docs
-      Term idTerm = getIdTerm(isNested? new BytesRef(cmd.getRootIdUsingRouteParam()): cmd.getIndexedId(), isNested);
+      Iterable<Document> nestedDocs = cmd.makeLuceneDocs();
+      Term idTerm = getIdTerm(cmd.getIndexedId());
       Term updateTerm = hasUpdateTerm ? cmd.updateTerm : idTerm;
-      if (isNested) {
-        log.debug("updateDocuments({})", cmd);
-        writer.updateDocuments(updateTerm, nestedDocs);
-      } else {
-        Document luceneDocument = cmd.getLuceneDocument();
-        log.debug("updateDocument({})", cmd);
-        writer.updateDocument(updateTerm, luceneDocument);
-      }
+
+      log.debug("updateDocuments({})", cmd);
+      writer.updateDocuments(updateTerm, nestedDocs);
 
       // If hasUpdateTerm, then delete any existing documents with the same ID other than the one added above
       //   (used in near-duplicate replacement)
@@ -988,8 +971,8 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
     }
   }
 
-  private Term getIdTerm(BytesRef termVal, boolean isNested) {
-    boolean useRootId = isNested || core.getLatestSchema().isUsableForChildDocs();
+  private Term getIdTerm(BytesRef termVal) {
+    boolean useRootId = core.getLatestSchema().isUsableForChildDocs();
     return new Term(useRootId ? IndexSchema.ROOT_FIELD_NAME : idField.getName(), termVal);
   }
 
diff --git a/solr/core/src/java/org/apache/solr/update/DocumentBuilder.java b/solr/core/src/java/org/apache/solr/update/DocumentBuilder.java
index aca8e85..56dca6d 100644
--- a/solr/core/src/java/org/apache/solr/update/DocumentBuilder.java
+++ b/solr/core/src/java/org/apache/solr/update/DocumentBuilder.java
@@ -135,6 +135,13 @@ public class DocumentBuilder {
     // Load fields from SolrDocument to Document
     for( SolrInputField field : doc ) {
 
+      // when in-place update, don't process the id & _root_; they won't change
+      if (forInPlaceUpdate) {
+        if (field.getName().equals(uniqueKeyFieldName) || field.getName().equals(IndexSchema.ROOT_FIELD_NAME)) {
+          continue;
+        }
+      }
+
       if (field.getFirstValue() instanceof SolrDocumentBase) {
         if (ignoreNestedDocs) {
           continue;
@@ -169,8 +176,7 @@ public class DocumentBuilder {
           hasField = true;
           if (sfield != null) {
             used = true;
-            addField(out, sfield, v,
-                     name.equals(uniqueKeyFieldName) ? false : forInPlaceUpdate);
+            addField(out, sfield, v, forInPlaceUpdate);
             // record the field as having a value
             usedFields.add(sfield.getName());
           }
@@ -178,34 +184,31 @@ public class DocumentBuilder {
           // Check if we should copy this field value to any other fields.
           // This could happen whether it is explicit or not.
           if (copyFields != null) {
-            // Do not copy this field if this document is to be used for an in-place update,
-            // and this is the uniqueKey field (because the uniqueKey can't change so no need to "update" the copyField).
-            if ( ! (forInPlaceUpdate && name.equals(uniqueKeyFieldName)) ) {
-              for (CopyField cf : copyFields) {
-                SchemaField destinationField = cf.getDestination();
-
-                final boolean destHasValues = usedFields.contains(destinationField.getName());
+            for (CopyField cf : copyFields) {
+              SchemaField destinationField = cf.getDestination();
 
-                // check if the copy field is a multivalued or not
-                if (!destinationField.multiValued() && destHasValues) {
-                  throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-                      "Multiple values encountered for non multiValued copy field " +
-                      destinationField.getName() + ": " + v);
-                }
+              final boolean destHasValues = usedFields.contains(destinationField.getName());
 
-                used = true;
+              // check if the copy field is a multivalued or not
+              if (!destinationField.multiValued() && destHasValues) {
+                throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+                    "Multiple values encountered for non multiValued copy field " +
+                    destinationField.getName() + ": " + v);
+              }
 
-                // Perhaps trim the length of a copy field
-                Object val = v;
-                if( val instanceof CharSequence && cf.getMaxChars() > 0 ) {
-                    val = cf.getLimitedValue(val.toString());
-                }
+              used = true;
 
-                addField(out, destinationField, val,
-                         destinationField.getName().equals(uniqueKeyFieldName) ? false : forInPlaceUpdate);
-                // record the field as having a value
-                usedFields.add(destinationField.getName());
+              // Perhaps trim the length of a copy field
+              Object val = v;
+              if( val instanceof CharSequence && cf.getMaxChars() > 0 ) {
+                  val = cf.getLimitedValue(val.toString());
               }
+
+              // TODO ban copyField populating uniqueKeyField; too problematic to support
+              addField(out, destinationField, val,
+                       destinationField.getName().equals(uniqueKeyFieldName) ? false : forInPlaceUpdate);
+              // record the field as having a value
+              usedFields.add(destinationField.getName());
             }
           }
         }
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateLog.java b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
index b928878..2b021f3 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateLog.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
@@ -69,6 +69,7 @@ import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestInfo;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.update.processor.DistributedUpdateProcessor;
 import org.apache.solr.update.processor.UpdateRequestProcessor;
@@ -102,6 +103,7 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private boolean debug = log.isDebugEnabled();
   private boolean trace = log.isTraceEnabled();
+  private boolean usableForChildDocs;
 
   // TODO: hack
   public FileSystem getFs() {
@@ -357,6 +359,8 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
 
     this.uhandler = uhandler;
 
+    usableForChildDocs = core.getLatestSchema().isUsableForChildDocs();
+
     if (dataDir.equals(lastDataDir)) {
       versionInfo.reload();
       core.getCoreMetricManager().registerMetricProducer(SolrInfoBean.Category.TLOG.toString(), this);
@@ -561,6 +565,13 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
     // TODO: we currently need to log to maintain correct versioning, rtg, etc
     // if ((cmd.getFlags() & UpdateCommand.REPLAY) != 0) return;
 
+    // This hack could be removed after SOLR-15064 when we insist updates to child docs include _root_.
+    // Until then, if we're in a buffering mode, then the solrDoc won't have the _root_ field.
+    // Otherwise, it should already be there, placed by the client.
+    if (usableForChildDocs && cmd.useRouteAsRoot != null && cmd.solrDoc.getField(IndexSchema.ROOT_FIELD_NAME) == null) {
+      cmd.solrDoc.setField(IndexSchema.ROOT_FIELD_NAME, cmd.getIndexedIdStr());
+    }
+
     synchronized (this) {
       if ((cmd.getFlags() & UpdateCommand.BUFFERING) != 0) {
         ensureBufferTlog();
@@ -685,6 +696,7 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
    * This may also be called when we updates are being buffered (from PeerSync/IndexFingerprint)
    */
   public void openRealtimeSearcher() {
+    log.debug("openRealtimeSearcher");
     synchronized (this) {
       // We must cause a new IndexReader to be opened before anything looks at these caches again
       // so that a cache miss will read fresh data.
diff --git a/solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java b/solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java
index 1dd993a..bb98036 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java
@@ -93,7 +93,52 @@ public class AtomicUpdateDocumentMerger {
     
     return false;
   }
-  
+
+  /**
+   * Merges the fromDoc into the toDoc using the atomic update syntax.
+   * This method will look for a nested document (possibly {@code toDoc} itself) with an
+   * equal ID, and merge into that one.
+   * @param sdoc the doc containing update instructions
+   * @param toDoc the target doc (possibly nested) before the update (will be modified in-place)
+   * @return toDoc with modifications; never null
+   */
+  public SolrInputDocument merge(SolrInputDocument sdoc, SolrInputDocument toDoc) {
+    if (mergeChildDocRecursive(sdoc, getRequiredId(sdoc), toDoc)) {
+      return toDoc;
+    }
+    throw new IllegalStateException("Did not find child ID " + getRequiredId(sdoc) +
+        " in parent ID " + getRequiredId(toDoc));
+  }
+
+  private boolean mergeChildDocRecursive(SolrInputDocument sdoc, Object sdocId, SolrInputDocument docWithChildren) {
+    if (sdocId.equals(getRequiredId(docWithChildren))) {
+      mergeDocHavingSameId(sdoc, docWithChildren);
+      return true;
+    }
+    for (SolrInputField inputField : docWithChildren) {
+      final Collection<Object> values = inputField.getValues();
+      if (values == null) {
+        continue;
+      }
+      for (Object value : values) {
+        if (isChildDoc(value)) {
+          if (mergeChildDocRecursive(sdoc, sdocId, (SolrInputDocument) value)) {
+            return true;
+          } // else continue the search
+        }
+      }
+    }
+    return false;
+  }
+
+  private String getRequiredId(SolrInputDocument sdoc) {
+    String id = schema.printableUniqueKey(sdoc);
+    if (id == null) {
+      throw new IllegalStateException("partial updates require that docs have an ID");
+    }
+    return id;
+  }
+
   /**
    * Merges the fromDoc into the toDoc using the atomic update syntax.
    * 
@@ -102,7 +147,7 @@ public class AtomicUpdateDocumentMerger {
    * @return toDoc with mutated values
    */
   @SuppressWarnings({"unchecked"})
-  public SolrInputDocument merge(final SolrInputDocument fromDoc, SolrInputDocument toDoc) {
+  private SolrInputDocument mergeDocHavingSameId(final SolrInputDocument fromDoc, SolrInputDocument toDoc) {
     for (SolrInputField sif : fromDoc.values()) {
      Object val = sif.getValue();
       if (val instanceof Map) {
@@ -195,6 +240,7 @@ public class AtomicUpdateDocumentMerger {
     for (String fieldName : sdoc.getFieldNames()) {
       Object fieldValue = sdoc.getField(fieldName).getValue();
       if (fieldName.equals(uniqueKeyFieldName)
+          || fieldName.equals(IndexSchema.ROOT_FIELD_NAME)
           || fieldName.equals(CommonParams.VERSION_FIELD)
           || fieldName.equals(routeFieldOrNull)) {
         if (fieldValue instanceof Map) {
@@ -339,15 +385,15 @@ public class AtomicUpdateDocumentMerger {
    */
   public boolean doInPlaceUpdateMerge(AddUpdateCommand cmd, Set<String> updatedFields) throws IOException {
     SolrInputDocument inputDoc = cmd.getSolrInputDocument();
-    BytesRef idBytes = cmd.getIndexedId();
+    BytesRef rootIdBytes = cmd.getIndexedId();
+    BytesRef idBytes = schema.indexableUniqueKey(cmd.getChildDocIdStr());
 
     updatedFields.add(CommonParams.VERSION_FIELD); // add the version field so that it is fetched too
     SolrInputDocument oldDocument = RealTimeGetComponent.getInputDocument
-      (cmd.getReq().getCore(), idBytes,
-       null, // don't want the version to be returned
-       updatedFields,
-       RealTimeGetComponent.Resolution.DOC);
-                                              
+      (cmd.getReq().getCore(), idBytes, rootIdBytes,
+          null, // don't want the version to be returned
+          updatedFields, RealTimeGetComponent.Resolution.DOC);
+
     if (oldDocument == RealTimeGetComponent.DELETED || oldDocument == null) {
       // This doc was deleted recently. In-place update cannot work, hence a full atomic update should be tried.
       return false;
@@ -385,8 +431,8 @@ public class AtomicUpdateDocumentMerger {
         partialDoc.addField(fieldName, oldDocument.getFieldValue(fieldName));
       }
     }
-    
-    merge(inputDoc, partialDoc);
+
+    mergeDocHavingSameId(inputDoc, partialDoc);
 
     // Populate the id field if not already populated (this can happen since stored fields were avoided during fetch from RTGC)
     if (!partialDoc.containsKey(schema.getUniqueKeyField().getName())) {
@@ -399,51 +445,6 @@ public class AtomicUpdateDocumentMerger {
     return true;
   }
 
-  /**
-   *
-   * Merges an Atomic Update inside a document hierarchy
-   * @param sdoc the doc containing update instructions
-   * @param oldDocWithChildren the doc (children included) before the update
-   * @param sdocWithChildren the updated doc prior to the update (children included)
-   * @return root doc (children included) after update
-   */
-  public SolrInputDocument mergeChildDoc(SolrInputDocument sdoc, SolrInputDocument oldDocWithChildren,
-                                         SolrInputDocument sdocWithChildren) {
-    // get path of document to be updated
-    String updatedDocPath = (String) sdocWithChildren.getFieldValue(IndexSchema.NEST_PATH_FIELD_NAME);
-    // get the SolrInputField containing the document which the AddUpdateCommand updates
-    SolrInputField sifToReplace = getFieldFromHierarchy(oldDocWithChildren, updatedDocPath);
-    // update SolrInputField, either appending or replacing the updated document
-    updateDocInSif(sifToReplace, sdocWithChildren, sdoc);
-    return oldDocWithChildren;
-  }
-
-  /**
-   *
-   * @param updateSif the SolrInputField to update its values
-   * @param cmdDocWChildren the doc to insert/set inside updateSif
-   * @param updateDoc the document that was sent as part of the Add Update Command
-   * @return updated SolrInputDocument
-   */
-  @SuppressWarnings({"unchecked"})
-  public SolrInputDocument updateDocInSif(SolrInputField updateSif, SolrInputDocument cmdDocWChildren, SolrInputDocument updateDoc) {
-    @SuppressWarnings({"rawtypes"})
-    List sifToReplaceValues = (List) updateSif.getValues();
-    final boolean wasList = updateSif.getValue() instanceof Collection;
-    int index = getDocIndexFromCollection(cmdDocWChildren, sifToReplaceValues);
-    SolrInputDocument updatedDoc = merge(updateDoc, cmdDocWChildren);
-    if(index == -1) {
-      sifToReplaceValues.add(updatedDoc);
-    } else {
-      sifToReplaceValues.set(index, updatedDoc);
-    }
-    // in the case where value was a List prior to the update and post update there is no more then one value
-    // it should be kept as a List.
-    final boolean singleVal = !wasList && sifToReplaceValues.size() <= 1;
-    updateSif.setValue(singleVal? sifToReplaceValues.get(0): sifToReplaceValues);
-    return cmdDocWChildren;
-  }
-
   protected void doSet(SolrInputDocument toDoc, SolrInputField sif, Object fieldVal) {
     String name = sif.getName();
     toDoc.setField(name, getNativeFieldValue(name, fieldVal));
@@ -690,21 +691,6 @@ public class AtomicUpdateDocumentMerger {
     }
   }
 
-  /**
-   *
-   * @param doc document to search for
-   * @param col collection of solrInputDocument
-   * @return index of doc in col, returns -1 if not found.
-   */
-  private static int getDocIndexFromCollection(SolrInputDocument doc, List<SolrInputDocument> col) {
-    for(int i = 0; i < col.size(); ++i) {
-      if(isDerivedFromDoc(col.get(i), doc)) {
-        return i;
-      }
-    }
-    return -1;
-  }
-
   private static Pair<String, Integer> getPathAndIndexFromNestPath(String nestPath) {
     List<String> splitPath = StrUtils.splitSmart(nestPath, '#');
     if(splitPath.size() == 1) {
diff --git a/solr/core/src/java/org/apache/solr/update/processor/ClassificationUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/ClassificationUpdateProcessor.java
index 8ce9814..750d418 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/ClassificationUpdateProcessor.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/ClassificationUpdateProcessor.java
@@ -34,6 +34,7 @@ import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.update.AddUpdateCommand;
+import org.apache.solr.update.DocumentBuilder;
 import org.apache.solr.update.processor.ClassificationUpdateProcessorFactory.Algorithm;
 
 /**
@@ -100,14 +101,13 @@ class ClassificationUpdateProcessor
   public void processAdd(AddUpdateCommand cmd)
       throws IOException {
     SolrInputDocument doc = cmd.getSolrInputDocument();
-    Document luceneDocument = cmd.getLuceneDocument();
-    String assignedClass;
     Object documentClass = doc.getFieldValue(trainingClassField);
     if (documentClass == null) {
+      Document luceneDocument = DocumentBuilder.toDocument(doc, cmd.getReq().getSchema(), false, true);
       List<ClassificationResult<BytesRef>> assignedClassifications = classifier.getClasses(luceneDocument, maxOutputClasses);
       if (assignedClassifications != null) {
         for (ClassificationResult<BytesRef> singleClassification : assignedClassifications) {
-          assignedClass = singleClassification.getAssignedClass().utf8ToString();
+          String assignedClass = singleClassification.getAssignedClass().utf8ToString();
           doc.addField(predictedClassField, assignedClass);
         }
       }
diff --git a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
index 70366b6..5b7800c 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
@@ -23,8 +23,6 @@ import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.annotations.VisibleForTesting;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.solr.client.solrj.SolrRequest;
@@ -49,7 +47,6 @@ import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.handler.component.RealTimeGetComponent;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.update.AddUpdateCommand;
 import org.apache.solr.update.CommitUpdateCommand;
@@ -497,13 +494,6 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
       // TODO: possibly set checkDeleteByQueries as a flag on the command?
       doLocalAdd(cmd);
 
-      // if the update updates a doc that is part of a nested structure,
-      // force open a realTimeSearcher to trigger a ulog cache refresh.
-      // This refresh makes RTG handler aware of this update.q
-      if(req.getSchema().isUsableForChildDocs() && shouldRefreshUlogCaches(cmd)) {
-        ulog.openRealtimeSearcher();
-      }
-
       if (clonedDoc != null) {
         cmd.solrDoc = clonedDoc;
       }
@@ -633,7 +623,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
    * @return AddUpdateCommand containing latest full doc at shard leader for the given id, or null if not found.
    */
   private UpdateCommand fetchFullUpdateFromLeader(AddUpdateCommand inplaceAdd, long versionOnUpdate) throws IOException {
-    String id = inplaceAdd.getPrintableId();
+    String id = inplaceAdd.getIndexedIdStr();
     UpdateShardHandler updateShardHandler = inplaceAdd.getReq().getCore().getCoreContainer().getUpdateShardHandler();
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.set(DISTRIB, false);
@@ -679,60 +669,47 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
   boolean getUpdatedDocument(AddUpdateCommand cmd, long versionOnUpdate) throws IOException {
     if (!AtomicUpdateDocumentMerger.isAtomicUpdate(cmd)) return false;
 
+    if (idField == null) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "Can't do atomic updates without a schema uniqueKeyField");
+    }
+
+    BytesRef rootIdBytes = cmd.getIndexedId(); // root doc; falls back to doc ID if no _route_
+    String rootDocIdString = cmd.getIndexedIdStr();
+
     Set<String> inPlaceUpdatedFields = AtomicUpdateDocumentMerger.computeInPlaceUpdatableFields(cmd);
     if (inPlaceUpdatedFields.size() > 0) { // non-empty means this is suitable for in-place updates
       if (docMerger.doInPlaceUpdateMerge(cmd, inPlaceUpdatedFields)) {
         return true;
-      } else {
-        // in-place update failed, so fall through and re-try the same with a full atomic update
-      }
+      } // in-place update failed, so fall through and re-try the same with a full atomic update
     }
-    
+
     // full (non-inplace) atomic update
-    SolrInputDocument sdoc = cmd.getSolrInputDocument();
-    BytesRef idBytes = cmd.getIndexedId();
-    String idString = cmd.getPrintableId();
-    SolrInputDocument oldRootDocWithChildren = RealTimeGetComponent.getInputDocument(cmd.getReq().getCore(), idBytes, RealTimeGetComponent.Resolution.ROOT_WITH_CHILDREN);
 
+    final SolrInputDocument oldRootDocWithChildren =
+        RealTimeGetComponent.getInputDocument(
+            req.getCore(),
+            rootIdBytes,
+            rootIdBytes,
+            null,
+            null,
+            RealTimeGetComponent.Resolution.ROOT_WITH_CHILDREN); // when no children, just fetches the doc
+
+    SolrInputDocument sdoc = cmd.getSolrInputDocument();
+    SolrInputDocument mergedDoc;
     if (oldRootDocWithChildren == null) {
-      if (versionOnUpdate > 0) {
+      if (versionOnUpdate > 0
+          || !rootDocIdString.equals(cmd.getChildDocIdStr())) {
         // could just let the optimistic locking throw the error
-        throw new SolrException(ErrorCode.CONFLICT, "Document not found for update.  id=" + idString);
-      } else if (req.getParams().get(ShardParams._ROUTE_) != null) {
-        // the specified document could not be found in this shard
-        // and was explicitly routed using _route_
-        throw new SolrException(ErrorCode.BAD_REQUEST,
-            "Could not find document id=" + idString +
-                ", perhaps the wrong \"_route_\" param was supplied");
+        throw new SolrException(ErrorCode.CONFLICT, "Document not found for update.  id=" + rootDocIdString);
       }
-    } else {
-      oldRootDocWithChildren.remove(CommonParams.VERSION_FIELD);
-    }
-
-
-    SolrInputDocument mergedDoc;
-    if(idField == null || oldRootDocWithChildren == null) {
       // create a new doc by default if an old one wasn't found
-      mergedDoc = docMerger.merge(sdoc, new SolrInputDocument());
+      mergedDoc = docMerger.merge(sdoc, new SolrInputDocument(idField.getName(), rootDocIdString));
     } else {
-      // Safety check: don't allow an update to an existing doc that has children, unless we actually support this.
-      if (req.getSchema().isUsableForChildDocs() // however, next line we see it doesn't support child docs
-          && req.getSchema().supportsPartialUpdatesOfChildDocs() == false
-          && req.getSearcher().count(new TermQuery(new Term(IndexSchema.ROOT_FIELD_NAME, idBytes))) > 1) {
-        throw new SolrException(ErrorCode.BAD_REQUEST, "This schema does not support partial updates to nested docs. See ref guide.");
-      }
+      oldRootDocWithChildren.remove(CommonParams.VERSION_FIELD);
 
-      String oldRootDocRootFieldVal = (String) oldRootDocWithChildren.getFieldValue(IndexSchema.ROOT_FIELD_NAME);
-      if(req.getSchema().savesChildDocRelations() && oldRootDocRootFieldVal != null &&
-          !idString.equals(oldRootDocRootFieldVal)) {
-        // this is an update where the updated doc is not the root document
-        SolrInputDocument sdocWithChildren = RealTimeGetComponent.getInputDocument(cmd.getReq().getCore(),
-            idBytes, RealTimeGetComponent.Resolution.DOC_WITH_CHILDREN);
-        mergedDoc = docMerger.mergeChildDoc(sdoc, oldRootDocWithChildren, sdocWithChildren);
-      } else {
-        mergedDoc = docMerger.merge(sdoc, oldRootDocWithChildren);
-      }
+      mergedDoc = docMerger.merge(sdoc, oldRootDocWithChildren);
     }
+
     cmd.solrDoc = mergedDoc;
     return true;
   }
@@ -1132,20 +1109,6 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
   }
 
   /**
-   *
-   * {@link AddUpdateCommand#isNested} is set in {@link org.apache.solr.update.processor.NestedUpdateProcessorFactory},
-   * which runs on leader and replicas just before run time processor
-   * @return whether this update changes a value of a nested document
-   */
-  private static boolean shouldRefreshUlogCaches(AddUpdateCommand cmd) {
-    // should be set since this method should only be called after DistributedUpdateProcessor#doLocalAdd,
-    // which runs post-processor in the URP chain, having NestedURP set cmd#isNested.
-    assert !cmd.getReq().getSchema().savesChildDocRelations() || cmd.isNested != null;
-    // true if update adds children
-    return Boolean.TRUE.equals(cmd.isNested);
-  }
-
-  /**
    * Returns a boolean indicating whether or not the caller should behave as
    * if this is the "leader" even when ZooKeeper is not enabled.  
    * (Even in non zk mode, tests may simulate updates to/from a leader)
diff --git a/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java
index 6f61f18..aa7c61e 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java
@@ -250,7 +250,7 @@ public class DistributedZkUpdateProcessor extends DistributedUpdateProcessor {
 
     if (isLeader && !isSubShardLeader)  {
       DocCollection coll = clusterState.getCollection(collection);
-      List<SolrCmdDistributor.Node> subShardLeaders = getSubShardLeaders(coll, cloudDesc.getShardId(), cmd.getRootIdUsingRouteParam(), cmd.getSolrInputDocument());
+      List<SolrCmdDistributor.Node> subShardLeaders = getSubShardLeaders(coll, cloudDesc.getShardId(), cmd.getIndexedIdStr(), cmd.getSolrInputDocument());
       // the list<node> will actually have only one element for an add request
       if (subShardLeaders != null && !subShardLeaders.isEmpty()) {
         ModifiableSolrParams params = new ModifiableSolrParams(filterParams(req.getParams()));
@@ -260,7 +260,7 @@ public class DistributedZkUpdateProcessor extends DistributedUpdateProcessor {
         params.set(DISTRIB_FROM_PARENT, cloudDesc.getShardId());
         cmdDistrib.distribAdd(cmd, subShardLeaders, params, true);
       }
-      final List<SolrCmdDistributor.Node> nodesByRoutingRules = getNodesByRoutingRules(clusterState, coll, cmd.getRootIdUsingRouteParam(), cmd.getSolrInputDocument());
+      final List<SolrCmdDistributor.Node> nodesByRoutingRules = getNodesByRoutingRules(clusterState, coll, cmd.getIndexedIdStr(), cmd.getSolrInputDocument());
       if (nodesByRoutingRules != null && !nodesByRoutingRules.isEmpty())  {
         ModifiableSolrParams params = new ModifiableSolrParams(filterParams(req.getParams()));
         params.set(DISTRIB_UPDATE_PARAM, DistribPhase.FROMLEADER.toString());
@@ -568,7 +568,7 @@ public class DistributedZkUpdateProcessor extends DistributedUpdateProcessor {
     zkCheck();
     if (cmd instanceof AddUpdateCommand) {
       AddUpdateCommand acmd = (AddUpdateCommand)cmd;
-      nodes = setupRequest(acmd.getRootIdUsingRouteParam(), acmd.getSolrInputDocument());
+      nodes = setupRequest(acmd.getIndexedIdStr(), acmd.getSolrInputDocument());
     } else if (cmd instanceof DeleteUpdateCommand) {
       DeleteUpdateCommand dcmd = (DeleteUpdateCommand)cmd;
       nodes = setupRequest(dcmd.getId(), null);
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 a6bb5d2..6826000 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
@@ -75,7 +75,7 @@ public class NestedUpdateProcessorFactory extends UpdateRequestProcessorFactory
     @Override
     public void processAdd(AddUpdateCommand cmd) throws IOException {
       SolrInputDocument doc = cmd.getSolrInputDocument();
-      cmd.isNested = processDocChildren(doc, null);
+      processDocChildren(doc, null);
       super.processAdd(cmd);
     }
 
diff --git a/solr/core/src/java/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactory.java b/solr/core/src/java/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactory.java
index f2f119b..a9a23f3 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactory.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactory.java
@@ -16,6 +16,10 @@
  */
 package org.apache.solr.update.processor;
 
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.Collections;
+
 import org.apache.lucene.util.BytesRef;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
@@ -32,10 +36,6 @@ import org.apache.solr.util.plugin.SolrCoreAware;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.Collections;
-
 import static org.apache.solr.common.SolrException.ErrorCode.SERVER_ERROR;
 import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
 
@@ -186,8 +186,13 @@ public class SkipExistingDocumentsProcessorFactory extends UpdateRequestProcesso
       assert null != indexedDocId;
 
       // we don't need any fields populated, we just need to know if the doc is in the tlog...
-      SolrInputDocument oldDoc = RealTimeGetComponent.getInputDocumentFromTlog(core, indexedDocId, null,
-                                                                               Collections.<String>emptySet(), false);
+      SolrInputDocument oldDoc =
+          RealTimeGetComponent.getInputDocumentFromTlog(
+              core,
+              indexedDocId,
+              null,
+              Collections.emptySet(),
+              RealTimeGetComponent.Resolution.PARTIAL);
       if (oldDoc == RealTimeGetComponent.DELETED) {
         return false;
       }
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 f7cab60..537009a 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
@@ -27,7 +27,7 @@
   <!-- for versioning -->
   <field name="_version_" type="long" indexed="false" stored="false" docValues="true"/>
   <!-- points to the root document of a block of nested documents -->
-  <field name="_root_" type="string" indexed="true" stored="true"/>
+  <field name="_root_" type="string" indexed="true" stored="false"/>
 
   <!-- populated by for NestedUpdateProcessor -->
   <field name="_nest_parent_" type="string" indexed="true" stored="true"/>
diff --git a/solr/core/src/test/org/apache/solr/cloud/NestedShardedAtomicUpdateTest.java b/solr/core/src/test/org/apache/solr/cloud/NestedShardedAtomicUpdateTest.java
index 43fc6fd..464fcdc 100644
--- a/solr/core/src/test/org/apache/solr/cloud/NestedShardedAtomicUpdateTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/NestedShardedAtomicUpdateTest.java
@@ -18,51 +18,62 @@
 package org.apache.solr.cloud;
 
 import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.lucene.util.IOUtils;
 import org.apache.solr.client.solrj.SolrClient;
 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.request.UpdateRequest;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-public class NestedShardedAtomicUpdateTest extends AbstractFullDistribZkTestBase {
-
-  public NestedShardedAtomicUpdateTest() {
-    stress = 0;
-    sliceCount = 4;
-    schemaString = "schema-nest.xml";
-  }
-
-  @Override
-  protected String getCloudSolrConfig() {
-    return "solrconfig-tlog.xml";
+public class NestedShardedAtomicUpdateTest extends SolrCloudTestCase { // used to extend AbstractFullDistribZkTestBase
+  private static final String DEFAULT_COLLECTION = "col1";
+  private static CloudSolrClient cloudClient;
+  private static List<SolrClient> clients; // not CloudSolrClient
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    configureCluster(1)
+        .addConfig("_default", configset("cloud-minimal"))
+        .configure();
+    // replace schema.xml with schema-test.xml
+    Path schemaPath = Paths.get(TEST_HOME()).resolve("collection1").resolve("conf").resolve("schema-nest.xml");
+    cluster.getZkClient().setData("/configs/_default/schema.xml", schemaPath.toFile(),true);
+
+    cloudClient = cluster.getSolrClient();
+    cloudClient.setDefaultCollection(DEFAULT_COLLECTION);
+
+    CollectionAdminRequest.createCollection(DEFAULT_COLLECTION, 4, 1)
+        .process(cloudClient);
+
+    clients = new ArrayList<>();
+    ClusterState clusterState = cloudClient.getClusterStateProvider().getClusterState();
+    for (Replica replica : clusterState.getCollection(DEFAULT_COLLECTION).getReplicas()) {
+      clients.add(getHttpSolrClient(replica.getCoreUrl()));
+    }
   }
 
-  @Override
-  protected String getCloudSchemaFile() {
-    return "schema-nest.xml";
+  @AfterClass
+  public static void afterClass() throws Exception {
+    IOUtils.close(clients);
   }
 
   @Test
-  @ShardsFixed(num = 4)
-  public void test() throws Exception {
-    boolean testFinished = false;
-    try {
-      sendWrongRouteParam();
-      doNestedInplaceUpdateTest();
-      doRootShardRoutingTest();
-      testFinished = true;
-    } finally {
-      if (!testFinished) {
-        printLayoutOnTearDown = true;
-      }
-    }
-  }
-
   public void doRootShardRoutingTest() throws Exception {
     assertEquals(4, cloudClient.getZkStateReader().getClusterState().getCollection(DEFAULT_COLLECTION).getSlices().size());
     final String[] ids = {"3", "4", "5", "6"};
@@ -117,11 +128,12 @@ public class NestedShardedAtomicUpdateTest extends AbstractFullDistribZkTestBase
       List<SolrDocument> grandChildren = (List) childDoc.getFieldValues("grandChildren");
       assertEquals(idIndex + 1, grandChildren.size());
       SolrDocument grandChild = grandChildren.get(0);
-      assertEquals(idIndex + 1, grandChild.getFirstValue("inplace_updatable_int"));
       assertEquals("3", grandChild.getFieldValue("id"));
+      assertEquals(idIndex + 1, grandChild.getFirstValue("inplace_updatable_int"));
     }
   }
 
+  @Test
   public void doNestedInplaceUpdateTest() throws Exception {
     assertEquals(4, cloudClient.getZkStateReader().getClusterState().getCollection(DEFAULT_COLLECTION).getSlices().size());
     final String[] ids = {"3", "4", "5", "6"};
@@ -147,39 +159,82 @@ public class NestedShardedAtomicUpdateTest extends AbstractFullDistribZkTestBase
 
     indexDocAndRandomlyCommit(aClient, params, doc);
 
+    int id1InPlaceCounter = 0;
+    int id2InPlaceCounter = 0;
+    int id3InPlaceCounter = 0;
     for (int fieldValue = 1; fieldValue < 5; ++fieldValue) {
-      doc = sdoc("id", "3", "inplace_updatable_int", map("inc", "1"));
+      // randomly increment a field on a root, middle, and leaf doc
+      if (random().nextBoolean()) {
+        id1InPlaceCounter++;
+        indexDoc(
+            getRandomSolrClient(),
+            params,
+            sdoc("id", "1", "inplace_updatable_int", map("inc", "1")));
+      }
+      if (random().nextBoolean()) {
+        id2InPlaceCounter++;
+        indexDoc(
+            getRandomSolrClient(),
+            params,
+            sdoc("id", "2", "inplace_updatable_int", map("inc", "1")));
+      }
+      if (random().nextBoolean()) {
+        id3InPlaceCounter++;
+        indexDoc(
+            getRandomSolrClient(),
+            params, // add root merely to show it doesn't interfere
+            sdoc("id", "3", "_root_", "1", "inplace_updatable_int", map("inc", "1")));
+      }
+      if (random().nextBoolean()) {
+        getRandomSolrClient().commit();
+      }
 
-      indexDocAndRandomlyCommit(getRandomSolrClient(), params, doc);
+      if (random().nextBoolean()) {
+        // assert RTG request respects _route_ param
+        QueryResponse routeRsp = getRandomSolrClient().query(params("qt","/get", "id","2", "_route_", "1"));
+        SolrDocument results = (SolrDocument) routeRsp.getResponse().get("doc");
+        assertNotNull("RTG should find doc because _route_ was set to the root documents' ID", results);
+        assertEquals("2", results.getFieldValue("id"));
+      }
 
-      // assert RTG request respects _route_ param
-      QueryResponse routeRsp = getRandomSolrClient().query(params("qt","/get", "id","2", "_route_", "1"));
-      SolrDocument results = (SolrDocument) routeRsp.getResponse().get("doc");
-      assertNotNull("RTG should find doc because _route_ was set to the root documents' ID", results);
-      assertEquals("2", results.getFieldValue("id"));
+      if (random().nextBoolean()) {
+        // assert all docs are indexed under the same root
+        assertEquals(0, getRandomSolrClient().query(params("q", "-_root_:1")).getResults().size());
+      }
 
-      // assert all docs are indexed under the same root
-      getRandomSolrClient().commit();
-      assertEquals(0, getRandomSolrClient().query(params("q", "-_root_:1")).getResults().size());
+      if (random().nextBoolean()) {
+        // assert all docs are indexed inside the same block
+        QueryResponse rsp = getRandomSolrClient().query(params("qt","/get", "id","1", "fl", "*, [child]"));
+        SolrDocument val = (SolrDocument) rsp.getResponse().get("doc");
+        assertEquals("1", val.getFieldValue("id"));
+        assertInplaceCounter(id1InPlaceCounter, val);
+        @SuppressWarnings({"unchecked"})
+        List<SolrDocument> children = (List) val.getFieldValues("children");
+        assertEquals(1, children.size());
+        SolrDocument childDoc = children.get(0);
+        assertEquals("2", childDoc.getFieldValue("id"));
+        assertInplaceCounter(id2InPlaceCounter, childDoc);
+        @SuppressWarnings({"unchecked"})
+        List<SolrDocument> grandChildren = (List) childDoc.getFieldValues("grandChildren");
+        assertEquals(1, grandChildren.size());
+        SolrDocument grandChild = grandChildren.get(0);
+        assertEquals("3", grandChild.getFieldValue("id"));
+        assertInplaceCounter(id3InPlaceCounter, grandChild);
+      }
+    }
+  }
 
-      // assert all docs are indexed inside the same block
-      QueryResponse rsp = getRandomSolrClient().query(params("qt","/get", "id","1", "fl", "*, [child]"));
-      SolrDocument val = (SolrDocument) rsp.getResponse().get("doc");
-      assertEquals("1", val.getFieldValue("id"));
-      @SuppressWarnings({"unchecked"})
-      List<SolrDocument> children = (List) val.getFieldValues("children");
-      assertEquals(1, children.size());
-      SolrDocument childDoc = children.get(0);
-      assertEquals("2", childDoc.getFieldValue("id"));
-      @SuppressWarnings({"unchecked"})
-      List<SolrDocument> grandChildren = (List) childDoc.getFieldValues("grandChildren");
-      assertEquals(1, grandChildren.size());
-      SolrDocument grandChild = grandChildren.get(0);
-      assertEquals(fieldValue, grandChild.getFirstValue("inplace_updatable_int"));
-      assertEquals("3", grandChild.getFieldValue("id"));
+  private void assertInplaceCounter(int expected, SolrDocument val) {
+    Number result = (Number) val.getFirstValue("inplace_updatable_int");
+    if (expected == 0) {
+      assertNull(val.toString(), result);
+    } else {
+      assertNotNull(val.toString(), result);
+      assertEquals(expected, result.intValue());
     }
   }
 
+  @Test
   public void sendWrongRouteParam() throws Exception {
     assertEquals(4, cloudClient.getZkStateReader().getClusterState().getCollection(DEFAULT_COLLECTION).getSlices().size());
     final String rootId = "1";
@@ -192,11 +247,11 @@ public class NestedShardedAtomicUpdateTest extends AbstractFullDistribZkTestBase
     int which = (rootId.hashCode() & 0x7fffffff) % clients.size();
     SolrClient aClient = clients.get(which);
 
-    indexDocAndRandomlyCommit(aClient, params("wt", "json", "_route_", rootId), doc, false);
+    indexDocAndRandomlyCommit(aClient, params("wt", "json", "_route_", rootId), doc);
 
     final SolrInputDocument childDoc = sdoc("id", rootId, "children", map("add", sdocs(sdoc("id", "2", "level_s", "child"))));
 
-    indexDocAndRandomlyCommit(aClient, rightParams, childDoc, false);
+    indexDocAndRandomlyCommit(aClient, rightParams, childDoc);
 
     final SolrInputDocument grandChildDoc = sdoc("id", "2", "grandChildren",
         map("add", sdocs(
@@ -209,29 +264,28 @@ public class NestedShardedAtomicUpdateTest extends AbstractFullDistribZkTestBase
         "wrong \"_route_\" param should throw an exception",
         () -> indexDocAndRandomlyCommit(aClient, wrongRootParams, grandChildDoc)
     );
-
-    assertTrue("message should suggest the wrong \"_route_\" param was supplied",
-        e.getMessage().contains("perhaps the wrong \"_route_\" param was supplied"));
+    assertTrue(e.toString(), e.getMessage().contains("Document not found for update"));
   }
 
   private void indexDocAndRandomlyCommit(SolrClient client, SolrParams params, SolrInputDocument sdoc) throws IOException, SolrServerException {
-    indexDocAndRandomlyCommit(client, params, sdoc, true);
-  }
-
-  private void indexDocAndRandomlyCommit(SolrClient client, SolrParams params, SolrInputDocument sdoc, boolean compareToControlCollection) throws IOException, SolrServerException {
-    if (compareToControlCollection) {
-      indexDoc(client, params, sdoc);
-    } else {
-      add(client, params, sdoc);
-    }
+    indexDoc(client, params, sdoc);
     // randomly commit docs
     if (random().nextBoolean()) {
       client.commit();
     }
   }
 
+  private void indexDoc(SolrClient client, SolrParams params, SolrInputDocument sdoc) throws IOException, SolrServerException {
+    final UpdateRequest updateRequest = new UpdateRequest();
+    updateRequest.add(sdoc);
+    updateRequest.setParams(new ModifiableSolrParams(params));
+    updateRequest.process(client, null);
+  }
+
   private SolrClient getRandomSolrClient() {
-    return clients.get(random().nextInt(clients.size()));
+    // randomly return one of these clients, to include the cloudClient
+    final int index = random().nextInt(clients.size() + 1);
+    return index == clients.size() ? cloudClient : clients.get(index);
   }
 
 }
diff --git a/solr/core/src/test/org/apache/solr/update/processor/AtomicUpdatesTest.java b/solr/core/src/test/org/apache/solr/update/processor/AtomicUpdatesTest.java
index a0c1402..3eba208 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/AtomicUpdatesTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/AtomicUpdatesTest.java
@@ -18,7 +18,6 @@ package org.apache.solr.update.processor;
 
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.Date;
 import java.util.List;
 
@@ -1331,36 +1330,6 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
         "/response/docs/[0]/single_i_dvn==5");
   }
 
-  /**
-   * Test what happens if we try to update the parent of a doc with children.
-   * This fails because _root_ is not stored which is currently required for doing this.
-   */
-  @Test
-  public void testUpdateNestedDocUnsupported() throws Exception {
-    assertU(adoc(sdoc(
-        "id", "1",
-        "children", Arrays.asList(sdoc(
-            "id", "100",
-            "cat", "childCat1")
-        )
-    )));
-
-    assertU(commit());
-
-    // update the parent doc to have a category
-    try {
-      assertU(adoc(sdoc(
-          "id", "1",
-          "cat", Collections.singletonMap("add", Arrays.asList("parentCat"))
-      )));
-      fail("expected a failure");
-    } catch (Exception e) {
-      assertEquals("org.apache.solr.common.SolrException: " +
-          "This schema does not support partial updates to nested docs. See ref guide.", e.toString());
-    }
-
-  }
-
   @Test
   public void testInvalidOperation() {
     SolrInputDocument doc;
diff --git a/solr/core/src/test/org/apache/solr/update/processor/NestedAtomicUpdateTest.java b/solr/core/src/test/org/apache/solr/update/processor/NestedAtomicUpdateTest.java
index d01ce72..853cb78 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/NestedAtomicUpdateTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/NestedAtomicUpdateTest.java
@@ -21,12 +21,14 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Iterator;
 import java.util.List;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
 import org.apache.lucene.util.BytesRef;
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.SolrInputField;
 import org.apache.solr.core.SolrCore;
@@ -35,6 +37,9 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import static org.apache.solr.handler.component.RealTimeGetComponent.Resolution.DOC;
+import static org.apache.solr.handler.component.RealTimeGetComponent.Resolution.ROOT_WITH_CHILDREN;
+
 public class NestedAtomicUpdateTest extends SolrTestCaseJ4 {
 
   private final static String VERSION = "_version_";
@@ -170,8 +175,7 @@ public class NestedAtomicUpdateTest extends SolrTestCaseJ4 {
 
 
     assertQ(req("q", "_root_:1", "fl", "*", "rows", "11"),
-        "//*[@numFound='11']",
-        "*[count(//str[@name='_root_'][.='1'])=11]"
+        "//*[@numFound='11']"
     );
 
     assertQ(req("q", "string_s:child", "fl", "*"),
@@ -182,8 +186,15 @@ public class NestedAtomicUpdateTest extends SolrTestCaseJ4 {
     // ensure updates work when block has more than 10 children
     for(int i = 10; i < 20; ++i) {
       docs = IntStream.range(i * 10, (i * 10) + 5).mapToObj(x -> sdoc("id", String.valueOf(x), "string_s", "grandChild")).collect(Collectors.toList());
-      doc = sdoc("id", String.valueOf(i), "grandChildren", Collections.singletonMap("add", docs));
-      addAndGetVersion(doc, params("wt", "json"));
+      doc =
+          sdoc(
+              "id",
+              String.valueOf(i),
+              "_root_", //shows we can specify the root field here instead of params
+              "1",
+              "grandChildren",
+              Collections.singletonMap("add", docs));
+      addAndGetVersion(doc, params("wt", "json")); // no _route_ with root
       assertU(commit());
     }
 
@@ -224,7 +235,7 @@ public class NestedAtomicUpdateTest extends SolrTestCaseJ4 {
 
     doc = sdoc("id", "2",
         "grandChild", Collections.singletonMap("add", sdocs(sdoc("id", "4", "child_s", "grandChild"), sdoc("id", "5", "child_s", "grandChild"))));
-    addAndGetVersion(doc, params("wt", "json"));
+    addAndGetVersion(doc, params("wt", "json", "_route_", "1"));
 
     assertU(commit());
 
@@ -286,15 +297,16 @@ public class NestedAtomicUpdateTest extends SolrTestCaseJ4 {
   @Test
   public void testBlockAtomicAdd() throws Exception {
 
+    final SolrInputDocument sdoc2 = sdoc("id", "2", "cat_ss", "child");
     SolrInputDocument doc = sdoc("id", "1",
         "cat_ss", new String[] {"aaa", "ccc"},
-        "child1", sdoc("id", "2", "cat_ss", "child")
+        "child1", sdoc2
     );
     assertU(adoc(doc));
 
     BytesRef rootDocId = new BytesRef("1");
     SolrCore core = h.getCore();
-    SolrInputDocument block = RealTimeGetComponent.getInputDocument(core, rootDocId, RealTimeGetComponent.Resolution.ROOT_WITH_CHILDREN);
+    SolrInputDocument block = RealTimeGetComponent.getInputDocument(core, rootDocId, rootDocId, null, null, ROOT_WITH_CHILDREN);
     // assert block doc has child docs
     assertTrue(block.containsKey("child1"));
 
@@ -305,10 +317,10 @@ public class NestedAtomicUpdateTest extends SolrTestCaseJ4 {
     // commit the changes
     assertU(commit());
 
-    SolrInputDocument committedBlock = RealTimeGetComponent.getInputDocument(core, rootDocId, RealTimeGetComponent.Resolution.ROOT_WITH_CHILDREN);
     BytesRef childDocId = new BytesRef("2");
-    // ensure the whole block is returned when resolveBlock is true and id of a child doc is provided
-    assertEquals(committedBlock.toString(), RealTimeGetComponent.getInputDocument(core, childDocId, RealTimeGetComponent.Resolution.ROOT_WITH_CHILDREN).toString());
+    assertEquals(sdoc2.toString(), removeSpecialFields(
+        RealTimeGetComponent.getInputDocument(core, childDocId, rootDocId, null, null, DOC)
+    ).toString());
 
     assertJQ(req("q","id:1")
         ,"/response/numFound==1"
@@ -359,7 +371,7 @@ public class NestedAtomicUpdateTest extends SolrTestCaseJ4 {
     //add greatGrandChild
     doc = sdoc("id", "4",
         "child4", Collections.singletonMap("add", sdoc("id", "5", "cat_ss", "greatGrandChild")));
-    addAndGetVersion(doc, params("wt", "json"));
+    addAndGetVersion(doc, params("wt", "json", "_route_", "1"));
 
     assertJQ(req("qt","/get", "id","1", "fl","id, cat_ss, child1, child2, child3, child4, [child]")
         ,"=={'doc':{'id':'1'" +
@@ -378,7 +390,7 @@ public class NestedAtomicUpdateTest extends SolrTestCaseJ4 {
     //add another greatGrandChild
     doc = sdoc("id", "4",
         "child4", Collections.singletonMap("add", sdoc("id", "6", "cat_ss", "greatGrandChild")));
-    addAndGetVersion(doc, params("wt", "json"));
+    addAndGetVersion(doc, params("wt", "json", "_route_", "1"));
 
     assertU(commit());
 
@@ -432,15 +444,16 @@ public class NestedAtomicUpdateTest extends SolrTestCaseJ4 {
 
   @Test
   public void testBlockAtomicSet() throws Exception {
+    SolrInputDocument sdoc2 = sdoc("id", "2", "cat_ss", "child");
     SolrInputDocument doc = sdoc("id", "1",
         "cat_ss", new String[] {"aaa", "ccc"},
-        "child1", Collections.singleton(sdoc("id", "2", "cat_ss", "child"))
+        "child1", Collections.singleton(sdoc2)
     );
     assertU(adoc(doc));
 
     BytesRef rootDocId = new BytesRef("1");
     SolrCore core = h.getCore();
-    SolrInputDocument block = RealTimeGetComponent.getInputDocument(core, rootDocId, RealTimeGetComponent.Resolution.ROOT_WITH_CHILDREN);
+    SolrInputDocument block = RealTimeGetComponent.getInputDocument(core, rootDocId, rootDocId, null, null, ROOT_WITH_CHILDREN);
     // assert block doc has child docs
     assertTrue(block.containsKey("child1"));
 
@@ -451,10 +464,10 @@ public class NestedAtomicUpdateTest extends SolrTestCaseJ4 {
     // commit the changes
     assertU(commit());
 
-    SolrInputDocument committedBlock = RealTimeGetComponent.getInputDocument(core, rootDocId, RealTimeGetComponent.Resolution.ROOT_WITH_CHILDREN);
     BytesRef childDocId = new BytesRef("2");
-    // ensure the whole block is returned when resolveBlock is true and id of a child doc is provided
-    assertEquals(committedBlock.toString(), RealTimeGetComponent.getInputDocument(core, childDocId, RealTimeGetComponent.Resolution.ROOT_WITH_CHILDREN).toString());
+    assertEquals(sdoc2.toString(), removeSpecialFields(
+        RealTimeGetComponent.getInputDocument(core, childDocId, rootDocId, null, null, DOC)
+    ).toString());
 
     assertJQ(req("q","id:1")
         ,"/response/numFound==1"
@@ -568,15 +581,16 @@ public class NestedAtomicUpdateTest extends SolrTestCaseJ4 {
 
   @Test
   public void testBlockAtomicRemove() throws Exception {
+    SolrInputDocument sdoc2 = sdoc("id", "2", "cat_ss", "child");
     SolrInputDocument doc = sdoc("id", "1",
         "cat_ss", new String[] {"aaa", "ccc"},
-        "child1", sdocs(sdoc("id", "2", "cat_ss", "child"), sdoc("id", "3", "cat_ss", "child"))
+        "child1", sdocs(sdoc2, sdoc("id", "3", "cat_ss", "child"))
     );
     assertU(adoc(doc));
 
     BytesRef rootDocId = new BytesRef("1");
     SolrCore core = h.getCore();
-    SolrInputDocument block = RealTimeGetComponent.getInputDocument(core, rootDocId, RealTimeGetComponent.Resolution.ROOT_WITH_CHILDREN);
+    SolrInputDocument block = RealTimeGetComponent.getInputDocument(core, rootDocId, rootDocId, null, null, ROOT_WITH_CHILDREN);
     // assert block doc has child docs
     assertTrue(block.containsKey("child1"));
 
@@ -587,10 +601,10 @@ public class NestedAtomicUpdateTest extends SolrTestCaseJ4 {
     // commit the changes
     assertU(commit());
 
-    SolrInputDocument committedBlock = RealTimeGetComponent.getInputDocument(core, rootDocId, RealTimeGetComponent.Resolution.ROOT_WITH_CHILDREN);
     BytesRef childDocId = new BytesRef("2");
-    // ensure the whole block is returned when resolveBlock is true and id of a child doc is provided
-    assertEquals(committedBlock.toString(), RealTimeGetComponent.getInputDocument(core, childDocId, RealTimeGetComponent.Resolution.ROOT_WITH_CHILDREN).toString());
+    assertEquals(sdoc2.toString(), removeSpecialFields(
+        RealTimeGetComponent.getInputDocument(core, childDocId, rootDocId, null, null, DOC)
+    ).toString());
 
     assertJQ(req("q","id:1")
         ,"/response/numFound==1"
@@ -655,15 +669,15 @@ public class NestedAtomicUpdateTest extends SolrTestCaseJ4 {
   private void testBlockAtomicSetToNullOrEmpty(boolean empty) throws Exception {
     // latlon field is included to ensure reading from LatLonDocValuesField is working due to atomic update.
     // See SOLR-13966 for further details.
+    SolrInputDocument sdoc2 = sdoc("id", "2", "cat_ss", "child");
     SolrInputDocument doc = sdoc("id", "1", "latlon", "0,0",
         "cat_ss", new String[] {"aaa", "ccc"},
-        "child1", sdocs(sdoc("id", "2", "cat_ss", "child"), sdoc("id", "3", "cat_ss", "child")));
+        "child1", sdocs(sdoc2, sdoc("id", "3", "cat_ss", "child")));
     assertU(adoc(doc));
 
     BytesRef rootDocId = new BytesRef("1");
     SolrCore core = h.getCore();
-    SolrInputDocument block = RealTimeGetComponent.getInputDocument(core, rootDocId,
-        RealTimeGetComponent.Resolution.ROOT_WITH_CHILDREN);
+    SolrInputDocument block = RealTimeGetComponent.getInputDocument(core, rootDocId, rootDocId, null, null, ROOT_WITH_CHILDREN);
     // assert block doc has child docs
     assertTrue(block.containsKey("child1"));
 
@@ -672,12 +686,10 @@ public class NestedAtomicUpdateTest extends SolrTestCaseJ4 {
     // commit the changes
     assertU(commit());
 
-    SolrInputDocument committedBlock = RealTimeGetComponent.getInputDocument(core, rootDocId,
-        RealTimeGetComponent.Resolution.ROOT_WITH_CHILDREN);
     BytesRef childDocId = new BytesRef("2");
-    // ensure the whole block is returned when resolveBlock is true and id of a child doc is provided
-    assertEquals(committedBlock.toString(), RealTimeGetComponent
-        .getInputDocument(core, childDocId, RealTimeGetComponent.Resolution.ROOT_WITH_CHILDREN).toString());
+    assertEquals(sdoc2.toString(), removeSpecialFields(
+        RealTimeGetComponent.getInputDocument(core, childDocId, rootDocId, null, null, DOC)
+    ).toString());
 
     assertJQ(req("q", "id:1"), "/response/numFound==1");
 
@@ -714,6 +726,32 @@ public class NestedAtomicUpdateTest extends SolrTestCaseJ4 {
         "/response/docs/[0]/cat_ss/[1]==\"ccc\"");
   }
 
+  public void testIncorrectlyUpdateChildDoc() throws Exception {
+    SolrInputDocument doc = sdoc("id", "1",
+        "child", sdoc("id", "2"));
+    assertU(adoc(doc));
+    assertU(commit());
+
+    // did not add _root_ like we should have
+    SolrException e = expectThrows(SolrException.class, () -> {
+      addAndGetVersion(
+          sdoc("id", "2", "grandchild", Collections.singletonMap("set", sdoc("id", "3"))), null);
+    });
+    assertTrue(e.toString(), e.getMessage().contains("Attempted an atomic/partial update to a " +
+        "child doc without indicating the _root_ somehow."));
+  }
+
+  private SolrInputDocument removeSpecialFields(SolrInputDocument doc) {
+    final Iterator<SolrInputField> fieldIter = doc.iterator();
+    while (fieldIter.hasNext()) {
+      SolrInputField field =  fieldIter.next();
+      if (field.getName().matches("^_.*_$")) {
+        fieldIter.remove();
+      }
+    }
+    return doc;
+  }
+
   @SuppressWarnings({"unchecked"})
   private static void assertDocContainsSubset(SolrInputDocument subsetDoc, SolrInputDocument fullDoc) {
     for(SolrInputField field: subsetDoc) {
diff --git a/solr/solr-ref-guide/src/indexing-nested-documents.adoc b/solr/solr-ref-guide/src/indexing-nested-documents.adoc
index c1bff8e..4e81587 100644
--- a/solr/solr-ref-guide/src/indexing-nested-documents.adoc
+++ b/solr/solr-ref-guide/src/indexing-nested-documents.adoc
@@ -20,9 +20,12 @@
 
 Solr supports indexing nested documents, described here, and ways to <<searching-nested-documents.adoc#searching-nested-documents,search and retrieve>> them very efficiently.
 
-By way of examples: nested documents in Solr can be used to bind a blog post (parent document) with comments (child documents) -- or as a way to model major product lines as parent documents, with multiple types of child documents representing individual SKUs (with unique sizes / colors) and supporting documention (either directly nested under the products, or under individual SKUs.
+By way of examples: nested documents in Solr can be used to bind a blog post (parent document)
+with comments (child documents) -- or as a way to model major product lines as parent documents,
+with multiple types of child documents representing individual SKUs (with unique sizes / colors) and supporting documentation (either directly nested under the products, or under individual SKUs.
 
-The "top most" parent with all children is referred to as a "root level" document or "block document" and it explains some of the nomenclature of related features.
+The "top most" parent with all children is referred to as a "root" document or formerly "block
+document" and it explains some of the nomenclature of related features.
 
 At query time, the <<other-parsers.adoc#block-join-query-parsers,Block Join Query Parsers>> can search these relationships,
  and the `<<transforming-result-documents.adoc#child-childdoctransformerfactory,[child]>>` Document Transformer can attach child (or other "descendent") documents to the result documents.
@@ -36,17 +39,25 @@ Nested documents may be indexed via either the XML or JSON data syntax, and is a
 [CAUTION]
 ====
 .Re-Indexing Considerations
-With the exception of in-place updates, <<#maintaining-integrity-with-updates-and-deletes,blocks of nested documents must be updated/deleted together>>.  Modifying or replacing individual child documents requires reindexing of the entire block (either explicitly/externally, or under the covers inside of Solr).  For some applications this may result in a lot of extra indexing overhead and may not be worth the performance gains at query time.
+With the exception of in-place updates, Solr must internally re-index an entire nested document tree
+if there are updates to it.  For some applications this may
+result in a lot of extra indexing overhead that may not be worth the performance gains at query
+time versus other modeling approaches.
 ====
 
+In the examples on this page, the IDs of child documents are always provided.  However, you need not
+generate such IDs; you can let Solr populate them automatically.  It will concatenate the ID of its
+parent with a separator and path information that should be unique.  Try it out for yourself!
+
 [#example-indexing-syntax]
 == Example Indexing Syntax: Psuedo-Fields
 
-This example shows what it looks like to index two root level "product" documents, each containing two different types of child documents specified in "psuedo-fields": "skus" and "manuals".  Two of the "sku" type documents have their own nested child "manuals" documents...
+This example shows what it looks like to index two root "product" documents, each containing two
+different types of child documents specified in "psuedo-fields": "skus" and "manuals".  Two of the "sku" type documents have their own nested child "manuals" documents...
 
 [NOTE]
 ====
-Even though the child documents in these examples are provided syntactically as field values syntactically, this is simply a matter of syntax and as such `skus` and `manuals` are not actual fields in the documents.  Consequently, these field names need not be defined in the schema and probably shouldn't be as it would be confusing.  There is no "child document" field type.
+Even though the child documents in these examples are provided syntactically as field values, this is simply a matter of syntax and as such `skus` and `manuals` are not actual fields in the documents.  Consequently, these field names need not be defined in the schema and probably shouldn't be as it would be confusing.  There is no "child document" field type.
 ====
 
 //
@@ -218,53 +229,81 @@ Indexing nested documents _requires_ an indexed field named `\_root_`:
 
 [source,xml]
 ----
-<field name="_root_" type="string" indexed="true" />
+<field name="_root_" type="string" indexed="true" stored="false" docValues="false" />
 ----
 
-Solr automatically populates this field in every nested document with the `id` value of the top most parent document in the block.
-
+* Solr automatically populates this field in _all_ documents with the `id` value of it's root document
+-- it's highest ancestor, possibly itself.
+* This field must be indexed (`indexed="true"`) but doesn't need to
+be either stored (`stored="true"`) or use doc values (`docValues="true"`), however you are free
+to do so if you find it useful.  If you want to use `uniqueBlock(\_root_)`
+<<json-facet-api#stat-facet-functions,field type limitation>>, then you should enable docValues.
 
-There are several additional schema considerations that should be considered for people who wish to use nested documents:
+Preferably, you will also define `\_nest_path_` which adds features and ease-of-use:
 
-* Nested child documents are very much documents in their own right even if certain nested documents hold different information from the parent, Therefore:
-** All field names in the schema can only be configured in one -- different types of child documents can not have the same field name configured in different ways.
-** It may be infeasible to use `required` for any field names that aren't required for all types of documents.
-** Even child documents need a _globally_ unique `id`.
-* `\_root_` must be configured to either be stored (`stored="true"`) or use doc values (`docValues="true"`) to enable <<updating-parts-of-documents#updating-child-documents,atomic updates of nested documents>>.
-** Also, beware of `uniqueBlock(\_root_)` <<json-facet-api#stat-facet-functions,field type limitation>>, if you plan to use one.
-* `\_nest_path_` is an optional field that (if defined) will be populated by Solr automatically with the ancestor path of each non-root document.
-+
 [source,xml]
 ----
 <fieldType name="_nest_path_" class="solr.NestPathField" />
 <field name="_nest_path_" type="_nest_path_" />`
 ----
-** This field is necessary if you wish to use <<updating-parts-of-documents#updating-child-documents,atomic updates of nested documents>>
-** This field is necessary in order for Solr to properly record & reconstruct the nested relationship of documents when using the `<<searching-nested-documents.adoc#child-doc-transformer,[child]>>` doc transformer.
-*** If this field does not exist, the `[child]` transformer will return all descendent child documents as a flattened list -- just as if they had been <<#indexing-anonymous-children,indexed as anonymous children>>.
-** If you do not use `\_nest_path_` it is strongly recommended that every document have some field that differentiates root documents from their nested children -- and differentiates different "types" of child documents.  This is not strictly necessary, so long as it's possible to write a "filter" query that can be used to isolate and select only parent documents for use in the <<other-parsers.adoc#block-join-query-parsers,block join query parsers>> and <<searching-nested-documents.adoc# [...]
-* `\_nest_parent_` is an optional field that (if defined) will be populated by Solr automatically to store the `id` of each document's _immediate_ parent document (if there is one).
-+
+
+* Solr automatically populates this field for any child document but not root documents.
+* This field enables Solr to properly record & reconstruct the named and nested relationship of documents
+when using the `<<searching-nested-documents.adoc#child-doc-transformer,[child]>>` doc transformer.
+** If this field does not exist, the `[child]` transformer will return all descendent child documents as a flattened list -- just as if they had been <<#indexing-anonymous-children,indexed as anonymous children>>.
+* If you do not use `\_nest_path_` it is strongly recommended that every document have some
+field that differentiates root documents from their nested children -- and differentiates different "types" of child documents.  This is not strictly necessary, so long as it's possible to write a "filter" query that can be used to isolate and select only parent documents for use in the <<other-parsers.adoc#block-join-query-parsers,block join query parsers>> and <<searching-nested-documents.adoc#child-doc-transformer,[child]>> doc transformer
+* It's possible to query on this field, although at present it's only documented how to in the
+context of `[child]`'s `childFilter` parameter.
+
+You might optionally want to define `\_nest_parent_` to store parent IDs:
+
 [source,xml]
 ----
 <field name="_nest_parent_" type="string" indexed="true" stored="true" />
 ----
 
+* Solr automatically populates this field in child documents but not root documents.
+
+
+Finally, understand that nested child documents are very much documents in their own right even if certain nested
+documents hold different information from the parent or other child documents, therefore:
+
+* All field names in the schema can only be configured in one -- different types of child documents can not have the same field name configured in different ways.
+* It may be infeasible to use `required` for any field names that aren't required for all types of
+documents.
+* Even child documents need a _globally_ unique `id`.
+
 [TIP]
 ====
-When using SolrCloud it is a _VERY_ good idea to use <<shards-and-indexing-data-in-solrcloud#document-routing,prefix based compositeIds>> with a common prefix for all documents in the block.  This makes it much easier to apply <<updating-parts-of-documents#updating-child-documents,atomic updates to individual child documents>>
+When using SolrCloud it is a _VERY_ good idea to use
+<<shards-and-indexing-data-in-solrcloud#document-routing,prefix based compositeIds>> with a
+common prefix for all documents in the nested document tree.  This makes it much easier to apply
+<<updating-parts-of-documents#updating-child-documents,atomic updates to individual child documents>>
 ====
 
 
 == Maintaining Integrity with Updates and Deletes
 
-Blocks of nested documents can be modified simply by adding/replacing the root document with more or fewer child/descendent documents as an application desires.  This can either be done explicitly/externally by an indexing client completely reindexing the root level document, or internally by Solr when a client uses <<updating-parts-of-documents#updating-child-documents,atomic updates>> to modify child documents.  This aspect isn't different than updating any normal document except that  [...]
+Nested document trees can be modified with Solr's
+<<updating-parts-of-documents#updating-child-documents,atomic/partial update>> feature to
+manipulate any document in a nested tree, and even to add new child documents.
+This aspect isn't different than updating any normal document -- Solr internally deletes the old
+nested document tree and it adds the newly modified one.
+Just be mindful to add a `_root_` field if the partial update is to a child doc so that Solr
+knows which Root doc it's related to.
 
-Clients should however be very careful to *never* add a root document that has the same `id` of a child document -- or vice-versa.  Solr does not prevent clients from attempting this, but *_it will violate integrity assumptions that Solr expects._*
+Solr demands that the `id` of _all_ documents in a collection be unique.  Solr enforces this for
+root documents within a shard but it doesn't for child documents to avoid the expense of checking.
+Clients should be very careful to *never* violate this.
 
-To delete an entire block of documents, you can simply delete-by-ID using the `id` of the root document.  Delete-by-ID will not work with the `id` of a child document, since only root document IDs are considered. (Instead, use <<updating-parts-of-documents#updating-child-documents,atomic updates>> to remove the child document from it's parent)
+To delete an entire nested document tree, you can simply delete-by-ID using the `id` of the root
+document.  Delete-by-ID will not work with the `id` of a child document, since only root document
+IDs are considered.  Instead, use delete-by-query (most efficient) or
+<<updating-parts-of-documents#updating-child-documents,atomic updates>> to remove the child document from it's parent.
 
-If you use Solr's delete-by-query APIs, you *MUST* be careful to ensure that any deletion query is structured to ensure no descendent children remain of any documents that are being deleted.  *_Doing otherwise will violate integrity assumptions that Solr expects._*
+If you use Solr's delete-by-query APIs, you *MUST* be careful to ensure that any deletion query
+is structured to ensure no descendent children remain of any documents that are being deleted.  *_Doing otherwise will violate integrity assumptions that Solr expects._*
 
 
 
@@ -384,7 +423,11 @@ This simplified approach was common in older versions of Solr, and can still be
 
 This approach should *NOT* be used when schemas include a `\_nest_path_` field, as the existence of that field triggers assumptions and changes in behavior in various query time functionality, such as the <<searching-nested-documents.adoc#child-doc-transformer,[child]>>, that will not work when nested documents do not have any intrinsic "nested path" information.
 
-The results of indexing anonymous nested children with a "Root-Only" schema are similar to what happens if you attempt to index "psuedo field" nested documents using a "Root-Only" schema.  Notably: since there is no nested path information for the <<searching-nested-documents.adoc#child-doc-transformer,[child]>> transformer to use to reconstruct the structured of a block of documents, it returns all matching children as a flat list, similar in structure to how they were originally indexed:
+The results of indexing anonymous nested children with a "Root-Only" schema are similar to what
+happens if you attempt to index "psuedo field" nested documents using a "Root-Only" schema.
+Notably: since there is no nested path information for the
+<<searching-nested-documents.adoc#child-doc-transformer,[child]>> transformer to use to reconstruct the structure of a nest
+of documents, it returns all matching children as a flat list, similar in structure to how they were originally indexed:
 
 
 
diff --git a/solr/solr-ref-guide/src/solr-upgrade-notes.adoc b/solr/solr-ref-guide/src/solr-upgrade-notes.adoc
index 751f87c..7db53cc 100644
--- a/solr/solr-ref-guide/src/solr-upgrade-notes.adoc
+++ b/solr/solr-ref-guide/src/solr-upgrade-notes.adoc
@@ -40,6 +40,20 @@ If you are upgrading from 7.x, see the section <<Upgrading from 7.x Releases>> b
 
 === Solr 8.8
 
+*Nested Documents*
+
+** When doing atomic/partial updates to a child document:
+*** Supply the `\_root_` field (the ID of the root document) so that Solr understands you are manipulating a child document and not a root document.
+In its absence, Solr looks at the `\_route_` parameter but that may go away because it's not an ideal substitute.
+If neither are present, Solr assumes you are updating a root document.
+If this assumption is false, Solr will do a cheap check that usually detects the problem and will
+throw an exception to alert you of the need to specify the Root ID.
+This backwards incompatible change was done to increase performance and robustness.
+*** This feature no longer requires stored=true or docValues=true on the `\_root_` field.  You might
+have it for other purposes though (e.g. for `uniqueBlock(...)`)
+*** This feature no longer requires the `\_nest_path_` field, although you probably ought to
+continue to define it as it's useful for other things.
+
 *Removed Contribs*
 
 * The search results clustering contrib has been removed from 8.x Solr line due to lack
diff --git a/solr/solr-ref-guide/src/updating-parts-of-documents.adoc b/solr/solr-ref-guide/src/updating-parts-of-documents.adoc
index f159d42..e15c1a0 100644
--- a/solr/solr-ref-guide/src/updating-parts-of-documents.adoc
+++ b/solr/solr-ref-guide/src/updating-parts-of-documents.adoc
@@ -105,19 +105,20 @@ The resulting document in our collection will be:
 
 === Updating Child Documents
 
-Solr supports modifying, adding and removing child documents as part of atomic updates.  Syntactically, updates changing the children of a document are very similar to a regular atomic updates of simle fields, as demonstrated by the examples below.
+Solr supports modifying, adding and removing child documents as part of atomic updates.
+Syntactically, updates changing the children of a document are very similar to regular atomic updates of simple fields, as demonstrated by the examples below.
 
-Schema and configuration requirements for updating child documents the same <<updating-parts-of-documents#field-storage,Field Storage>> requirements for atomic updates mentioned above, combined with the <<indexing-nested-documents#schema-configuration,schema configuration rules for Indexing Nested Documents>> -- notably:
-* The `\_root_` field must configured with `stored="true"` or `docValues="true"`
-* The `\_nest_path_` field must exist (it is implicitly `docValues="true"`)
+Schema and configuration requirements for updating child documents use the same
+<<updating-parts-of-documents#field-storage,Field Storage>> requirements for atomic updates mentioned above.
 
-Under the hood, When Solr processes atomic updates on nested documents, it retrieves the entire block structure (up to and including the common "Root" document), reindexes the structure after applying the atomic update, and deletes the old documents.
+Under the hood, Solr conceptually behaves similarly for nested documents as for non-nested documents, it's just that it applies to entire trees (from the root) of nested documents instead of stand-alone documents.  You can expect more overhead because of this.  In-place updates avoid that.
 
 [IMPORTANT]
 ====
 .Routing Updates using child document Ids in SolrCloud
 
-When SolrCloud recieves document updates, the <<shards-and-indexing-data-in-solrcloud#document-routing,document routing>> rules for the collection is used to determine which shard should process the update based on the `id` of the document.
+When SolrCloud receives document updates, the
+<<shards-and-indexing-data-in-solrcloud#document-routing,document routing>> rules for the collection is used to determine which shard should process the update based on the `id` of the document.
 
 When sending an update that specifies the `id` of a _child document_ this will not work by default: the correct shard to send the document to is based on the `id` of the "Root" document for the block the child document is in, *not* the `id` of the child document being updated.
 
@@ -126,7 +127,12 @@ Solr offers two solutions to address this:
 * Clients may specify a <<shards-and-indexing-data-in-solrcloud#document-routing,`\_route_` parameter>>, with the `id` of the Root document as the parameter value, on each update to tell Solr which shard should process the update.
 * Clients can use the (default) `compositeId` router's "prefix routing" feature when indexing all documents to ensure that all child/descendent documents in a Block use the same `id` prefix as the Root level document.  This will cause Solr's default routing logic to automatically send child document updates to the correct shard.
 
-All of the examples below use `id` prefixes, so no `\_route_` param will be neccessary for these examples.
+Furthermore, you _should_ (sometimes _must_) specify the Root document's ID in the `\_root_`
+field of this partial update.  This is how Solr understands that you are updating a child
+document, and not a Root document.  Without it, Solr only guesses that the `\_route_` param is
+equivalent, but it may be absent or not equivalent (e.g. when using the `implicit` router).
+
+All of the examples below use `id` prefixes, so no `\_route_` param will be necessary for these examples.
 ====
 
 For the upcoming examples, we'll assume an index containing the same documents covered in <<indexing-nested-documents#example-indexing-syntax,Indexing Nested Documents>>:
@@ -142,6 +148,7 @@ All of the <<#atomic-updates,Atomic Update operations>> mentioned above are supp
 curl -X POST 'http://localhost:8983/solr/gettingstarted/update?commit=true' -H 'Content-Type: application/json' --data-binary '[
 {
   "id": "P11!S31",
+  "_root_": "P11!prod",
   "price_i": { "inc": 73 },
   "color_s": { "set": "GREY" }
 } ]'
@@ -156,6 +163,7 @@ As with normal (multiValued) fields, the `set` keyword can be used to replace al
 curl -X POST 'http://localhost:8983/solr/gettingstarted/update?commit=true' -H 'Content-Type: application/json' --data-binary '[
 {
   "id": "P22!S22",
+  "_root_": "P22!prod",
   "manuals": { "set": [ { "id": "P22!D77",
                           "name_s": "Why Red Pens Are the Best",
                           "content_t": "... correcting papers ...",
@@ -177,6 +185,7 @@ As with normal (multiValued) fields, the `add` keyword can be used to add additi
 curl -X POST 'http://localhost:8983/solr/gettingstarted/update?commit=true' -H 'Content-Type: application/json' --data-binary '[
 {
   "id": "P11!S21",
+  "_root_": "P11!prod",
   "manuals": { "add": { "id": "P11!D99",
                         "name_s": "Why Red Staplers Are the Best",
                         "content_t": "Once upon a time, Mike Judge ...",
@@ -194,6 +203,7 @@ As with normal (multiValued) fields, the `remove` keyword can be used to remove
 curl -X POST 'http://localhost:8983/solr/gettingstarted/update?commit=true' -H 'Content-Type: application/json' --data-binary '[
 {
   "id": "P11!S21",
+  "_root_": "P11!prod",
   "manuals": { "remove": { "id": "P11!D41" } }
 } ]'
 ----
diff --git a/solr/solrj/src/java/org/apache/solr/common/SolrDocument.java b/solr/solrj/src/java/org/apache/solr/common/SolrDocument.java
index d7c04e9..77c28a5 100644
--- a/solr/solrj/src/java/org/apache/solr/common/SolrDocument.java
+++ b/solr/solrj/src/java/org/apache/solr/common/SolrDocument.java
@@ -25,6 +25,7 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.function.BiConsumer;
 
 import org.apache.solr.common.util.NamedList;
 
@@ -234,6 +235,33 @@ public class SolrDocument extends SolrDocumentBase<Object, SolrDocument> impleme
     return _fields.entrySet().iterator();
   }
 
+  /** Beta API; may change at will. */
+  // TODO SOLR-15063 reconcile SolrDocumentBase/SolrDocument/SolrInputDocument debacle
+  public void visitSelfAndNestedDocs(BiConsumer<String, SolrDocument> consumer) {
+    consumer.accept(null, this);
+    for (Entry<String, Object> keyVal : entrySet()) {
+      final Object value = keyVal.getValue();
+      if (value instanceof SolrDocument) {
+        consumer.accept(keyVal.getKey(), (SolrDocument) value);
+      } else if (value instanceof Collection) {
+        Collection<?> cVal = (Collection<?>) value;
+        for (Object v : cVal) {
+          if (v instanceof SolrDocument) {
+            consumer.accept(keyVal.getKey(), (SolrDocument) v);
+          } else {
+            break; // either they are all SolrDocs, or none are
+          }
+        }
+      }
+    }
+
+    if (_childDocuments != null) {
+      for (SolrDocument childDocument : _childDocuments) {
+        consumer.accept(null, childDocument);
+      }
+    }
+  }
+
   //-----------------------------------------------------------------------------------------
   // JSTL Helpers
   //-----------------------------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/SolrInputDocument.java b/solr/solrj/src/java/org/apache/solr/common/SolrInputDocument.java
index e13d78c..b6c7e1d 100644
--- a/solr/solrj/src/java/org/apache/solr/common/SolrInputDocument.java
+++ b/solr/solrj/src/java/org/apache/solr/common/SolrInputDocument.java
@@ -278,6 +278,33 @@ public class SolrInputDocument extends SolrDocumentBase<SolrInputField, SolrInpu
     }
   }
 
+  /** Beta API; may change at will. */
+  // TODO SOLR-15063 reconcile SolrDocumentBase/SolrDocument/SolrInputDocument debacle
+  public void visitSelfAndNestedDocs(BiConsumer<String, SolrInputDocument> consumer) {
+    consumer.accept(null, this);
+    for (SolrInputField field : values()) {
+      final Object value = field.getValue();
+      if (value instanceof SolrInputDocument) {
+        consumer.accept(field.name, (SolrInputDocument) value);
+      } else if (value instanceof Collection) {
+        Collection<?> cVal = (Collection<?>) value;
+        for (Object v : cVal) {
+          if (v instanceof SolrInputDocument) {
+            consumer.accept(field.name, (SolrInputDocument) v);
+          } else {
+            break; // either they are all solr docs, or none are
+          }
+        }
+      }
+    }
+
+    if (_childDocuments != null) {
+      for (SolrInputDocument childDocument : _childDocuments) {
+        consumer.accept(null, childDocument);
+      }
+    }
+  }
+
   /** Returns the list of child documents, or null if none. */
   public List<SolrInputDocument> getChildDocuments() {
     return _childDocuments;