You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by is...@apache.org on 2016/12/12 14:35:29 UTC

lucene-solr:jira/solr-5944: SOLR-5944: Fixing the schema default present/absent bug, removing special handling of dynamic fields

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/solr-5944 e2ced0b8e -> e528b46c6


SOLR-5944: Fixing the schema default present/absent bug, removing special handling of dynamic fields


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

Branch: refs/heads/jira/solr-5944
Commit: e528b46c6aa1824aa96f2a47a0d55f085967e02a
Parents: e2ced0b
Author: Ishan Chattopadhyaya <ic...@gmail.com>
Authored: Mon Dec 12 21:34:59 2016 +0700
Committer: Ishan Chattopadhyaya <ic...@gmail.com>
Committed: Mon Dec 12 21:34:59 2016 +0700

----------------------------------------------------------------------
 .../handler/component/RealTimeGetComponent.java | 27 ++++---
 .../apache/solr/search/SolrIndexSearcher.java   |  6 +-
 .../java/org/apache/solr/update/UpdateLog.java  | 18 +++--
 .../processor/AtomicUpdateDocumentMerger.java   | 81 +++++++-------------
 ...BasedVersionConstraintsProcessorFactory.java |  2 +-
 .../update/TestInPlaceUpdatesStandalone.java    |  8 +-
 .../org/apache/solr/update/UpdateLogTest.java   |  8 +-
 7 files changed, 76 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e528b46c/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
----------------------------------------------------------------------
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 31fff74..4e40cbf 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
@@ -236,7 +236,7 @@ public class RealTimeGetComponent extends SearchComponent
                  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 = (SolrDocument) resolveFullDocument(core, idBytes.get(), rsp.getReturnFields(), doc, entry);
+                 doc = (SolrDocument) resolveFullDocument(core, idBytes.get(), rsp.getReturnFields(), doc, entry, null);
                  if (doc == null) {
                    break;
                  }
@@ -361,11 +361,13 @@ 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.
    * @return Returns the merged document, i.e. the resolved full document, or null if the document was not found (deleted
    *          after the resolving began)
    */
   private static SolrDocument resolveFullDocument(SolrCore core, BytesRef idBytes,
-                                           ReturnFields returnFields, SolrDocumentBase partialDoc, List logEntry) throws IOException {
+                                           ReturnFields returnFields, SolrDocumentBase partialDoc, List logEntry, Set<String> onlyTheseFields) throws IOException {
     if (idBytes == null || logEntry.size() != 5) {
       throw new SolrException(ErrorCode.INVALID_STATE, "Either Id field not present in partial document or log entry doesn't have previous version.");
     }
@@ -374,10 +376,10 @@ public class RealTimeGetComponent extends SearchComponent
 
     // get the last full document from ulog
     UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
-    long lastPrevPointer = ulog.applyPartialUpdates(idBytes, prevPointer, prevVersion, partialDoc);
+    long lastPrevPointer = ulog.applyPartialUpdates(idBytes, prevPointer, prevVersion, onlyTheseFields, partialDoc);
 
     if (lastPrevPointer == -1) { // full document was not found in tlog, but exists in index
-      SolrDocument mergedDoc = mergePartialDocWithFullDocFromIndex(core, idBytes, returnFields, partialDoc);
+      SolrDocument mergedDoc = mergePartialDocWithFullDocFromIndex(core, idBytes, returnFields, onlyTheseFields, partialDoc);
       return mergedDoc;
     } 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
@@ -432,6 +434,8 @@ 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. 
@@ -439,7 +443,7 @@ public class RealTimeGetComponent extends SearchComponent
    *         document doesn't exist in the index.
    */
   private static SolrDocument mergePartialDocWithFullDocFromIndex(SolrCore core, BytesRef idBytes, ReturnFields returnFields,
-        SolrDocumentBase partialDoc) throws IOException {
+		  Set<String> onlyTheseFields, SolrDocumentBase partialDoc) throws IOException {
     RefCounted<SolrIndexSearcher> searcherHolder = core.getRealtimeSearcher(); //Searcher();
     try {
       // now fetch last document from index, and merge partialDoc on top of it
@@ -461,9 +465,11 @@ public class RealTimeGetComponent extends SearchComponent
         return doc;
       }
 
+      SolrDocument doc;
+      Set<String> decorateFields = onlyTheseFields == null ? searcher.getNonStoredDVs(false): onlyTheseFields; 
       Document luceneDocument = searcher.doc(docid, returnFields.getLuceneFieldNames());
-      SolrDocument doc = toSolrDoc(luceneDocument, core.getLatestSchema());
-      searcher.decorateDocValueFields(doc, docid, searcher.getNonStoredDVs(false));
+      doc = toSolrDoc(luceneDocument, core.getLatestSchema());
+      searcher.decorateDocValueFields(doc, docid, decorateFields);
 
       long docVersion = (long) doc.getFirstValue(DistributedUpdateProcessor.VERSION_FIELD);
       Object partialVersionObj = partialDoc.getFieldValue(DistributedUpdateProcessor.VERSION_FIELD);
@@ -493,7 +499,8 @@ public class RealTimeGetComponent extends SearchComponent
    *                  was an in-place update. In that case, should this partial document be resolved to a full document (by following
    *                  back prevPointer/prevVersion)?
    */
-  public static SolrInputDocument getInputDocumentFromTlog(SolrCore core, BytesRef idBytes, boolean resolveFullDocument) {
+  public static SolrInputDocument getInputDocumentFromTlog(SolrCore core, BytesRef idBytes,
+      Set<String> onlyTheseNonStoredDVs, boolean resolveFullDocument) {
 
     UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
 
@@ -514,7 +521,7 @@ public class RealTimeGetComponent extends SearchComponent
                 // 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 = toSolrInputDocument(
-                    resolveFullDocument(core, idBytes, new SolrReturnFields(), doc, entry),
+                    resolveFullDocument(core, idBytes, new SolrReturnFields(), doc, entry, onlyTheseNonStoredDVs),
                     core.getLatestSchema());
                 if (doc == null) {
                   return DELETED;
@@ -571,7 +578,7 @@ public class RealTimeGetComponent extends SearchComponent
     RefCounted<SolrIndexSearcher> searcherHolder = null;
     try {
       SolrIndexSearcher searcher = null;
-      sid = getInputDocumentFromTlog(core, idBytes, resolveFullDocument);
+      sid = getInputDocumentFromTlog(core, idBytes, onlyTheseNonStoredDVs, resolveFullDocument);
       if (sid == DELETED) {
         return null;
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e528b46c/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
index 6d13b51..895de7d 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
@@ -807,7 +807,11 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
           }
         }
       } else {
-        final DocValuesType dvType = fieldInfos.fieldInfo(fieldName).getDocValuesType();
+        FieldInfo fi = fieldInfos.fieldInfo(fieldName);
+        if (fi == null) {
+          continue; // Searcher doesn't have info about this field, hence ignore it.
+        }
+        final DocValuesType dvType = fi.getDocValuesType();
         switch (dvType) {
           case NUMERIC:
             final NumericDocValues ndv = leafReader.getNumericDocValues(fieldName);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e528b46c/solr/core/src/java/org/apache/solr/update/UpdateLog.java
----------------------------------------------------------------------
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 d51b8f0..3264508 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateLog.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
@@ -35,6 +35,7 @@ import java.util.List;
 import java.util.ListIterator;
 import java.util.Locale;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ExecutorCompletionService;
 import java.util.concurrent.Future;
 import java.util.concurrent.SynchronousQueue;
@@ -780,11 +781,14 @@ public static final int VERSION_IDX = 1;
    * @param id          Binary representation of the unique key field
    * @param prevPointer Pointer to the previous entry in the ulog, based on which the current in-place update was made.
    * @param prevVersion Version of the previous entry in the ulog, based on which the current in-place update was made.
+   * @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.
    * @param latestPartialDoc   Partial document that is to be populated
    * @return Returns 0 if a full document was found in the log, -1 if no full document was found. If full document was supposed
    * to be found in the tlogs, but couldn't be found (because the logs were rotated) then the prevPointer is returned.
    */
-  public long applyPartialUpdates(BytesRef id, long prevPointer, long prevVersion, SolrDocumentBase latestPartialDoc) {
+  public long applyPartialUpdates(BytesRef id, long prevPointer, long prevVersion,
+      Set<String> onlyTheseFields, SolrDocumentBase latestPartialDoc) {
     
     // nocommit: changes that should probably be made (see jira comments for elaboration) ...
     // 1) "final List<TransactionLog> lookupLogs" should be created once, outside of any looping
@@ -819,7 +823,7 @@ public static final int VERSION_IDX = 1;
       // if this is an ADD (i.e. full document update), stop here
       if ((flags & UpdateLog.ADD) == UpdateLog.ADD) {
         partialUpdateDoc = (SolrInputDocument) entry.get(entry.size() - 1);
-        applyOlderUpdates(latestPartialDoc, partialUpdateDoc);
+        applyOlderUpdates(latestPartialDoc, partialUpdateDoc, onlyTheseFields);
         return 0; // Full document was found in the tlog itself
       }
       if (entry.size() < 5) {
@@ -828,9 +832,13 @@ public static final int VERSION_IDX = 1;
       }
       // This update is an inplace update, get the partial doc. The input doc is always at last position.
       partialUpdateDoc = (SolrInputDocument) entry.get(entry.size() - 1);
-      applyOlderUpdates(latestPartialDoc, partialUpdateDoc);
+      applyOlderUpdates(latestPartialDoc, partialUpdateDoc, onlyTheseFields);
       prevPointer = (long) entry.get(UpdateLog.PREV_POINTER_IDX);
       prevVersion = (long) entry.get(UpdateLog.PREV_VERSION_IDX);
+      
+      if (onlyTheseFields != null && latestPartialDoc.keySet().containsAll(onlyTheseFields)) {
+        return 0; // all the onlyTheseFields have been resolved, safe to abort now.
+      }
     }
 
     return -1; // last full document is not supposed to be in tlogs, but it must be in the index
@@ -839,10 +847,10 @@ public static final int VERSION_IDX = 1;
   /**
    * Add all fields from olderDoc into newerDoc if not already present in newerDoc
    */
-  private void applyOlderUpdates(SolrDocumentBase newerDoc, SolrInputDocument olderDoc) {
+  private void applyOlderUpdates(SolrDocumentBase newerDoc, SolrInputDocument olderDoc, Set<String> mergeFields) {
     for (String fieldName : olderDoc.getFieldNames()) {
       // if the newerDoc has this field, then this field from olderDoc can be ignored
-      if (!newerDoc.containsKey(fieldName)) {
+      if (!newerDoc.containsKey(fieldName) && (mergeFields == null || mergeFields.contains(fieldName))) {
         for (Object val : olderDoc.getFieldValues(fieldName)) {
           newerDoc.addField(fieldName, val);
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e528b46c/solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java
----------------------------------------------------------------------
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 58554d5..afc8883 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
@@ -29,7 +29,6 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.Term;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.solr.common.SolrException;
@@ -234,36 +233,27 @@ public class AtomicUpdateDocumentMerger {
           return Collections.emptySet();
       }
 
-      // nocommit: why does it matter if this is a dynamic field?
-      //
-      // nocommit: comment below says dynamicField dests that don't yet exist won't work for inplace...
-      // nocommit: ...but that doesn't explain why <dynamicFields> are special
-      // nocommit: why would a <field> that doesn't yet exist in the IndexWriter work?
-      //
-      // nocommit: see elaboration of this question in jira comments
-      if (schema.isDynamicField(fieldName)) {
-        if (null == fieldNamesFromIndexWriter) { // lazy init fieldNamesFromIndexWriter
+      if (null == fieldNamesFromIndexWriter) { // lazy init fieldNamesFromIndexWriter
+        try {
+          SolrCore core = cmd.getReq().getCore();
+          RefCounted<IndexWriter> holder = core.getSolrCoreState().getIndexWriter(core);
           try {
-            SolrCore core = cmd.getReq().getCore();
-            RefCounted<IndexWriter> holder = core.getSolrCoreState().getIndexWriter(core);
-            try {
-              IndexWriter iw = holder.get();
-              fieldNamesFromIndexWriter = iw.getFieldNames();
-            } finally {
-              holder.decref();
-            }
-          } catch (IOException e) {
-            throw new RuntimeException(e); // nocommit
-
-            // nocommit: if we're going to throw a runtime excep it should be a SolrException with usefull code/msg
-            // nocommit: but why are we catching/wrapping the IOE?  why aren't we rethrowing?
+            IndexWriter iw = holder.get();
+            fieldNamesFromIndexWriter = iw.getFieldNames();
+          } finally {
+            holder.decref();
           }
-        }
-        if (! fieldNamesFromIndexWriter.contains(fieldName) ) {
-          // nocommit: this comment is not usefull - doesn't explain *WHY*
-          return Collections.emptySet(); // if dynamic field and this field doesn't exist, DV update can't work
+        } catch (IOException e) {
+          throw new RuntimeException(e); // nocommit
+
+          // nocommit: if we're going to throw a runtime excep it should be a SolrException with usefull code/msg
+          // nocommit: but why are we catching/wrapping the IOE?  why aren't we rethrowing?
         }
       }
+      if (! fieldNamesFromIndexWriter.contains(fieldName) ) {
+        // nocommit: this comment is not usefull - doesn't explain *WHY*
+        return Collections.emptySet(); // if dynamic field and this field doesn't exist, DV update can't work
+      }
     }
     return candidateFields;
   }
@@ -284,42 +274,29 @@ public class AtomicUpdateDocumentMerger {
 
     updatedFields.add(DistributedUpdateProcessor.VERSION_FIELD); // add the version field so that it is fetched too
     SolrInputDocument oldDocument = RealTimeGetComponent.getInputDocument(cmd.getReq().getCore(),
-                                              idBytes, true, updatedFields, false); // avoid stored fields from index
+                                              idBytes, true, updatedFields, true); // avoid stored fields from index
     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;
     }
 
-    // If oldDocument doesn't have a field that is present in updatedFields, 
-    // then fetch the field from RT searcher into oldDocument.
-    // This can happen if the oldDocument was fetched from tlog, but the DV field to be
-    // updated was not in that document.
-    if (oldDocument.getFieldNames().containsAll(updatedFields) == false) {
-      RefCounted<SolrIndexSearcher> searcherHolder = null;
-      try {
-        searcherHolder = cmd.getReq().getCore().getRealtimeSearcher();
-        SolrIndexSearcher searcher = searcherHolder.get();
-        int docid = searcher.getFirstMatch(new Term(idField.getName(), idBytes));
-        if (docid >= 0) {
-          searcher.decorateDocValueFields(oldDocument, docid, updatedFields);
-        } else {
-          // Not all fields needed for DV updates were found in the document obtained
-          // from tlog, and the document wasn't found in the index.
-          return false; // do a full atomic update
-        }
-      } finally {
-        if (searcherHolder != null) {
-          searcherHolder.decref();
-        }
-      }
-    }
-
     if (oldDocument.containsKey(DistributedUpdateProcessor.VERSION_FIELD) == false) {
       throw new SolrException (ErrorCode.INVALID_STATE, "There is no _version_ in previous document. id=" + 
           cmd.getPrintableId());
     }
     Long oldVersion = (Long) oldDocument.remove(DistributedUpdateProcessor.VERSION_FIELD).getValue();
 
+    // If the oldDocument contains any other field apart from updatedFields (or id/version field), then remove them.
+    // This can happen, despite requesting for these fields in the call to RTGC.getInputDocument, if the document was
+    // fetched from the tlog and had all these fields (possibly because it was a full document ADD operation).
+    if (updatedFields != null) {
+      Collection<String> names = new HashSet<String>(oldDocument.getFieldNames());
+      for (String fieldName: names) {
+        if (fieldName.equals(DistributedUpdateProcessor.VERSION_FIELD)==false && fieldName.equals("id")==false && updatedFields.contains(fieldName)==false) {
+          oldDocument.remove(fieldName);
+        }
+      }
+    }
     // Copy over all supported DVs from oldDocument to partialDoc
     //
     // Assuming multiple updates to the same doc: field 'dv1' in one update, then field 'dv2' in a second

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e528b46c/solr/core/src/java/org/apache/solr/update/processor/DocBasedVersionConstraintsProcessorFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/DocBasedVersionConstraintsProcessorFactory.java b/solr/core/src/java/org/apache/solr/update/processor/DocBasedVersionConstraintsProcessorFactory.java
index 1ac66e0..4eb3f79 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/DocBasedVersionConstraintsProcessorFactory.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/DocBasedVersionConstraintsProcessorFactory.java
@@ -266,7 +266,7 @@ public class DocBasedVersionConstraintsProcessorFactory extends UpdateRequestPro
         // nocommit: any call to getInputDocument* would by definition return a "full document"
         //
         // (last patch fixed getInputDocument call, but not this getInputDocumentFromTlog; intentional?)
-        oldDoc = RealTimeGetComponent.getInputDocumentFromTlog(core, indexedDocId, false);
+        oldDoc = RealTimeGetComponent.getInputDocumentFromTlog(core, indexedDocId, null, false);
         if (oldDoc == RealTimeGetComponent.DELETED) {
           return true;
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e528b46c/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java
index 5194954..a100184 100644
--- a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java
+++ b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java
@@ -983,7 +983,13 @@ public class TestInPlaceUpdatesStandalone extends TestRTGBase {
   public void testIsInPlaceUpdate() throws Exception {
     Set<String> inPlaceUpdatedFields = new HashSet<String>();
 
-    // In-place updates:
+    // In-place updateable field updated before it exists SHOULD NOT BE in-place updated:
+    inPlaceUpdatedFields = AtomicUpdateDocumentMerger.isInPlaceUpdate(
+        UpdateLogTest.getAddUpdate(null, sdoc("id", "1", "_version_", 42L, "inplace_updatable_float", map("set", 10))));
+    assertFalse(inPlaceUpdatedFields.contains("inplace_updatable_float"));
+
+    // In-place updateable field updated after it exists SHOULD BE in-place updated:
+    addAndGetVersion(sdoc("id", "1", "inplace_updatable_float", "0"), params()); // setting up the dv
     inPlaceUpdatedFields = AtomicUpdateDocumentMerger.isInPlaceUpdate(
         UpdateLogTest.getAddUpdate(null, sdoc("id", "1", "_version_", 42L, "inplace_updatable_float", map("set", 10))));
     assertTrue(inPlaceUpdatedFields.contains("inplace_updatable_float"));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e528b46c/solr/core/src/test/org/apache/solr/update/UpdateLogTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/UpdateLogTest.java b/solr/core/src/test/org/apache/solr/update/UpdateLogTest.java
index ccc55f9..43e860f 100644
--- a/solr/core/src/test/org/apache/solr/update/UpdateLogTest.java
+++ b/solr/core/src/test/org/apache/solr/update/UpdateLogTest.java
@@ -72,7 +72,7 @@ public class UpdateLogTest extends SolrTestCaseJ4 {
     assertEquals(3L, ((NumericDocValuesField)partialDoc.getFieldValue("val1_i_dvo")).numericValue());
     assertFalse(partialDoc.containsKey("title_s"));
 
-    long returnVal = ulog.applyPartialUpdates(cmd.getIndexedId(), prevPointer, prevVersion, partialDoc);
+    long returnVal = ulog.applyPartialUpdates(cmd.getIndexedId(), prevPointer, prevVersion, null, partialDoc);
 
     assertEquals(0, returnVal);
     assertEquals(1000, Integer.parseInt(partialDoc.getFieldValue("price").toString()));
@@ -96,7 +96,7 @@ public class UpdateLogTest extends SolrTestCaseJ4 {
     assertEquals(5L, ((NumericDocValuesField)partialDoc.getFieldValue("val1_i_dvo")).numericValue());
     assertFalse(partialDoc.containsKey("title_s"));
 
-    returnVal = ulog.applyPartialUpdates(cmd.getIndexedId(), prevPointer, prevVersion, partialDoc);
+    returnVal = ulog.applyPartialUpdates(cmd.getIndexedId(), prevPointer, prevVersion, null, partialDoc);
 
     assertEquals(0, returnVal);
     assertEquals(2000, Integer.parseInt(partialDoc.getFieldValue("price").toString()));
@@ -125,7 +125,7 @@ public class UpdateLogTest extends SolrTestCaseJ4 {
     assertEquals(6L, ((NumericDocValuesField)partialDoc.getFieldValue("val1_i_dvo")).numericValue());
     assertFalse(partialDoc.containsKey("title_s"));
 
-    long returnVal = ulog.applyPartialUpdates(cmd.getIndexedId(), prevPointer, prevVersion, partialDoc);
+    long returnVal = ulog.applyPartialUpdates(cmd.getIndexedId(), prevPointer, prevVersion, null, partialDoc);
 
     assertEquals(-1, returnVal);
   }
@@ -151,7 +151,7 @@ public class UpdateLogTest extends SolrTestCaseJ4 {
 
     // If an in-place update depends on a non-add (i.e. DBI), assert that an exception is thrown.
     SolrException ex = expectThrows(SolrException.class, () -> {
-        long returnVal = ulog.applyPartialUpdates(cmd.getIndexedId(), prevPointer, prevVersion, partialDoc);
+        long returnVal = ulog.applyPartialUpdates(cmd.getIndexedId(), prevPointer, prevVersion, null, partialDoc);
         fail("502 depends on 501, 501 depends on 500, but 500 is a"
              + " DELETE. This should've generated an exception. returnVal is: "+returnVal);
       });