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

lucene-solr:jira/solr-5944: SOLR-5944: fix test bugs related to SolrQueryReq/UpdateCommand lifecycles

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/solr-5944 d49c7a25f -> 020c74eea


SOLR-5944: fix test bugs related to SolrQueryReq/UpdateCommand lifecycles


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

Branch: refs/heads/jira/solr-5944
Commit: 020c74eeaeeccc1286d71b6dc198fda4a83b1485
Parents: d49c7a2
Author: Chris Hostetter <ho...@apache.org>
Authored: Mon Dec 12 20:28:01 2016 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Mon Dec 12 20:28:01 2016 -0700

----------------------------------------------------------------------
 .../update/TestInPlaceUpdatesStandalone.java    | 131 ++++++++++--------
 .../org/apache/solr/update/UpdateLogTest.java   | 135 ++++++++++---------
 2 files changed, 151 insertions(+), 115 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/020c74ee/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 ae7d07a..20f4462 100644
--- a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java
+++ b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java
@@ -19,6 +19,7 @@
 package org.apache.solr.update;
 
 import static org.junit.internal.matchers.StringContains.containsString;
+import static org.apache.solr.update.UpdateLogTest.buildAddUpdateCommand;
 
 import java.util.Arrays;
 import java.util.HashMap;
@@ -37,8 +38,8 @@ import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.SolrInputField;
-import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.update.processor.DistributedUpdateProcessor;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.search.SolrIndexSearcher;
@@ -975,64 +976,63 @@ public class TestInPlaceUpdatesStandalone extends TestRTGBase {
     assertEquals(102.0f, rtgDoc.getFieldValue("inplace_updatable_float"));
   }
 
-  /** @see AtomicUpdateDocumentMerger#isInPlaceUpdate */
+  /** 
+   * @see #callIsInPlaceUpdate
+   * @see AtomicUpdateDocumentMerger#isInPlaceUpdate 
+   */
   @Test
-  public void testIsInPlaceUpdate() throws Exception {
+  public void testIsInPlaceUpdate() throws Exception { // nocommit: rename when isInPlaceUpdate is renamed
     Set<String> inPlaceUpdatedFields = new HashSet<String>();
 
     // 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))));
+    inPlaceUpdatedFields = callIsInPlaceUpdate(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))));
+    inPlaceUpdatedFields = callIsInPlaceUpdate(sdoc("id", "1", "_version_", 42L,
+                                                    "inplace_updatable_float", map("set", 10)));
     assertTrue(inPlaceUpdatedFields.contains("inplace_updatable_float"));
 
-    inPlaceUpdatedFields.clear();
-    inPlaceUpdatedFields = AtomicUpdateDocumentMerger.isInPlaceUpdate(
-        UpdateLogTest.getAddUpdate(null, sdoc("id", "1", "_version_", 42L, "inplace_updatable_float", map("inc", 10))));
+    inPlaceUpdatedFields = callIsInPlaceUpdate(sdoc("id", "1", "_version_", 42L,
+                                                    "inplace_updatable_float", map("inc", 10)));
     assertTrue(inPlaceUpdatedFields.contains("inplace_updatable_float"));
-
-    inPlaceUpdatedFields.clear();
-    inPlaceUpdatedFields = AtomicUpdateDocumentMerger.isInPlaceUpdate(
-        UpdateLogTest.getAddUpdate(null, sdoc("id", "1", "_version_", 42L, "inplace_updatable_int", map("set", 10))));
+    
+    inPlaceUpdatedFields = callIsInPlaceUpdate(sdoc("id", "1", "_version_", 42L,
+                                                    "inplace_updatable_int", map("set", 10)));
     assertTrue(inPlaceUpdatedFields.contains("inplace_updatable_int"));
-
+    
     // Non in-place updates
-    inPlaceUpdatedFields.clear();
     addAndGetVersion(sdoc("id", "1", "stored_i", "0"), params()); // setting up the dv
-    assertTrue("stored field updated", AtomicUpdateDocumentMerger.isInPlaceUpdate(
-        UpdateLogTest.getAddUpdate(null, sdoc("id", "1", "_version_", 42L, "stored_i", map("inc", 1)))).isEmpty());
-
-    assertTrue("No map means full document update", AtomicUpdateDocumentMerger.isInPlaceUpdate(
-        UpdateLogTest.getAddUpdate(null, sdoc("id", "1", "_version_", 42L, "inplace_updatable_int", "100"))).isEmpty());
-
+    assertTrue("stored field updated",
+               callIsInPlaceUpdate(sdoc("id", "1", "_version_", 42L, "stored_i", map("inc", 1))).isEmpty());
+    
+    assertTrue("No map means full document update",
+               callIsInPlaceUpdate(sdoc("id", "1", "_version_", 42L,
+                                        "inplace_updatable_int", "100")).isEmpty());
+  
     assertTrue("non existent dynamic dv field updated first time",
-        AtomicUpdateDocumentMerger.isInPlaceUpdate(
-            UpdateLogTest.getAddUpdate(null, sdoc("id", "1", "_version_", 42L, "new_updateable_int_i_dvo", map("set", 10)))).isEmpty());
-
+               callIsInPlaceUpdate(sdoc("id", "1", "_version_", 42L,
+                                        "new_updateable_int_i_dvo", map("set", 10))).isEmpty());
+    
     // After adding a full document with the dynamic dv field, in-place update should work
     addAndGetVersion(sdoc("id", "2", "new_updateable_int_i_dvo", "0"), params()); // setting up the dv
-    if (random().nextBoolean())
+    if (random().nextBoolean()) {
       assertU(commit("softCommit", "false"));
-    inPlaceUpdatedFields.clear();
-    inPlaceUpdatedFields = AtomicUpdateDocumentMerger.isInPlaceUpdate(
-        UpdateLogTest.getAddUpdate(null, sdoc("id", "2", "_version_", 42L, "new_updateable_int_i_dvo", map("set", 10))));
+    }
+    inPlaceUpdatedFields = callIsInPlaceUpdate(sdoc("id", "2", "_version_", 42L,
+                                                    "new_updateable_int_i_dvo", map("set", 10)));
     assertTrue(inPlaceUpdatedFields.contains("new_updateable_int_i_dvo"));
 
     // If a supported dv field has a copyField target which is supported, it should be an in-place update
-    inPlaceUpdatedFields = AtomicUpdateDocumentMerger.isInPlaceUpdate
-      (UpdateLogTest.getAddUpdate(null, sdoc("id", "1", "_version_", 42L,
-                                             "copyfield1_src__both_updateable", map("set", 10))));
+    inPlaceUpdatedFields = callIsInPlaceUpdate(sdoc("id", "1", "_version_", 42L,
+                                                    "copyfield1_src__both_updateable", map("set", 10)));
     assertTrue(inPlaceUpdatedFields.contains("copyfield1_src__both_updateable"));
 
     // If a supported dv field has a copyField target which is not supported, it should not be an in-place update
-    inPlaceUpdatedFields = AtomicUpdateDocumentMerger.isInPlaceUpdate
-      (UpdateLogTest.getAddUpdate(null, sdoc("id", "1", "_version_", 42L,
-                                             "copyfield2_src__only_src_updatable", map("set", 10))));
+    inPlaceUpdatedFields = callIsInPlaceUpdate(sdoc("id", "1", "_version_", 42L,
+                                                    "copyfield2_src__only_src_updatable", map("set", 10)));
     assertTrue(inPlaceUpdatedFields.isEmpty());
   }
 
@@ -1052,27 +1052,50 @@ public class TestInPlaceUpdatesStandalone extends TestRTGBase {
     version1 = addAndAssertVersion(version1, "id", "1", "inplace_updatable_float", map("set", 200));
 
     // Test the AUDM.doInPlaceUpdateMerge() method is working fine
-    AddUpdateCommand cmd = UpdateLogTest.getAddUpdate(null, sdoc("id", "1", "_version_", 42L, "inplace_updatable_float", map("inc", 10)));
-    SolrQueryRequest req = new LocalSolrQueryRequest(h.getCore(), params());
-    AtomicUpdateDocumentMerger docMerger = new AtomicUpdateDocumentMerger(req);
-    boolean done = docMerger.doInPlaceUpdateMerge(cmd, AtomicUpdateDocumentMerger.isInPlaceUpdate(cmd));
-    assertTrue(done);
-    assertEquals(42L, cmd.getSolrInputDocument().getFieldValue("_version_"));
-    assertEquals(42L, cmd.getSolrInputDocument().getFieldValue("_version_"));
-    assertEquals(210f, cmd.getSolrInputDocument().getFieldValue("inplace_updatable_float"));
-    assertFalse(cmd.getSolrInputDocument().containsKey("title_s")); // in-place merged doc shouldn't have non-inplace fields from the index/tlog
-    assertEquals(version1, cmd.prevVersion);
-
+    try (SolrQueryRequest req = req()) {
+      AddUpdateCommand cmd = buildAddUpdateCommand(req, sdoc("id", "1", "_version_", 42L,
+                                                             "inplace_updatable_float", map("inc", 10)));
+      AtomicUpdateDocumentMerger docMerger = new AtomicUpdateDocumentMerger(req);
+      assertTrue(docMerger.doInPlaceUpdateMerge(cmd, AtomicUpdateDocumentMerger.isInPlaceUpdate(cmd)));
+      assertEquals(42L, cmd.getSolrInputDocument().getFieldValue("_version_"));
+      assertEquals(42L, cmd.getSolrInputDocument().getFieldValue("_version_"));
+      assertEquals(210f, cmd.getSolrInputDocument().getFieldValue("inplace_updatable_float"));
+      // in-place merged doc shouldn't have non-inplace fields from the index/tlog
+      assertFalse(cmd.getSolrInputDocument().containsKey("title_s"));
+      assertEquals(version1, cmd.prevVersion);
+    }
+    
     // do a commit, and the same results should be repeated
     assertU(commit("softCommit", "false"));
 
-    cmd = UpdateLogTest.getAddUpdate(null, sdoc("id", "1", "_version_", 42L, "inplace_updatable_float", map("inc", 10)));
-    done = docMerger.doInPlaceUpdateMerge(cmd, AtomicUpdateDocumentMerger.isInPlaceUpdate(cmd));
-    assertTrue(done);
-    assertEquals(42L, cmd.getSolrInputDocument().getFieldValue("_version_"));
-    assertEquals(42L, cmd.getSolrInputDocument().getFieldValue("_version_"));
-    assertEquals(210f, cmd.getSolrInputDocument().getFieldValue("inplace_updatable_float"));
-    assertFalse(cmd.getSolrInputDocument().containsKey("title_s")); // in-place merged doc shouldn't have non-inplace fields from the index/tlog
-    assertEquals(version1, cmd.prevVersion);
+    // Test the AUDM.doInPlaceUpdateMerge() method is working fine
+    try (SolrQueryRequest req = req()) {
+      AddUpdateCommand cmd = buildAddUpdateCommand(req, sdoc("id", "1", "_version_", 42L,
+                                                             "inplace_updatable_float", map("inc", 10)));
+      AtomicUpdateDocumentMerger docMerger = new AtomicUpdateDocumentMerger(req);
+      assertTrue(docMerger.doInPlaceUpdateMerge(cmd, AtomicUpdateDocumentMerger.isInPlaceUpdate(cmd)));
+      assertEquals(42L, cmd.getSolrInputDocument().getFieldValue("_version_"));
+      assertEquals(42L, cmd.getSolrInputDocument().getFieldValue("_version_"));
+      assertEquals(210f, cmd.getSolrInputDocument().getFieldValue("inplace_updatable_float"));
+      // in-place merged doc shouldn't have non-inplace fields from the index/tlog
+      assertFalse(cmd.getSolrInputDocument().containsKey("title_s")); 
+      assertEquals(version1, cmd.prevVersion);
+    }
+  }
+  
+  /** 
+   * Helper method that sets up a req/cmd to run {@link AtomicUpdateDocumentMerger#isInPlaceUpdate} 
+   * on the specified solr input document.
+   */
+  private static Set<String> callIsInPlaceUpdate(final SolrInputDocument sdoc) throws Exception {
+    // nocommit: rename when isInPlaceUpdate is renamed
+
+    try (SolrQueryRequest req = req()) {
+      AddUpdateCommand cmd = new AddUpdateCommand(req);
+      cmd.solrDoc = sdoc;
+      assertTrue(cmd.solrDoc.containsKey(DistributedUpdateProcessor.VERSION_FIELD));
+      cmd.setVersion(Long.parseLong(cmd.solrDoc.getFieldValue(DistributedUpdateProcessor.VERSION_FIELD).toString()));
+      return AtomicUpdateDocumentMerger.isInPlaceUpdate(cmd);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/020c74ee/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 43e860f..0e8d86e 100644
--- a/solr/core/src/test/org/apache/solr/update/UpdateLogTest.java
+++ b/solr/core/src/test/org/apache/solr/update/UpdateLogTest.java
@@ -19,12 +19,12 @@ package org.apache.solr.update;
 import java.util.List;
 
 import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.util.BytesRef;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.handler.component.RealTimeGetComponent;
-import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.update.processor.DistributedUpdateProcessor;
 import org.junit.BeforeClass;
@@ -33,6 +33,10 @@ import static org.junit.internal.matchers.StringContains.containsString;
 
 public class UpdateLogTest extends SolrTestCaseJ4 {
 
+  /** BytesRef that can be re-used to lookup doc with id "1" */
+  private static final BytesRef DOC_1_INDEXED_ID = new BytesRef("1");
+
+
   static UpdateLog ulog = null;
 
   @BeforeClass
@@ -57,13 +61,11 @@ public class UpdateLogTest extends SolrTestCaseJ4 {
    */
   public void testApplyPartialUpdatesOnMultipleInPlaceUpdatesInSequence() {    
     // Add a full update, two in-place updates and verify applying partial updates is working
-    AddUpdateCommand cmd;
-    cmd = getAddUpdate(null, sdoc("id", "1", "title_s", "title1", "val1_i_dvo", "1", "_version_", "100"));
-    ulog.add(cmd);
-    ulog.add(getAddUpdate(100L, sdoc("id", "1", "price", "1000", "val1_i_dvo", "2", "_version_", "101")));
-    ulog.add(getAddUpdate(101L, sdoc("id", "1", "val1_i_dvo", "3", "_version_", "102")));
+    ulogAdd(ulog, null, sdoc("id", "1", "title_s", "title1", "val1_i_dvo", "1", "_version_", "100"));
+    ulogAdd(ulog, 100L, sdoc("id", "1", "price", "1000", "val1_i_dvo", "2", "_version_", "101"));
+    ulogAdd(ulog, 101L, sdoc("id", "1", "val1_i_dvo", "3", "_version_", "102"));
 
-    Object partialUpdate = ulog.lookup(cmd.getIndexedId());
+    Object partialUpdate = ulog.lookup(DOC_1_INDEXED_ID);
     SolrDocument partialDoc = RealTimeGetComponent.toSolrDoc((SolrInputDocument)((List)partialUpdate).get(4), 
         h.getCore().getLatestSchema(), true);
     long prevVersion = (Long)((List)partialUpdate).get(3);
@@ -72,7 +74,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, null, partialDoc);
+    long returnVal = ulog.applyPartialUpdates(DOC_1_INDEXED_ID, prevPointer, prevVersion, null, partialDoc);
 
     assertEquals(0, returnVal);
     assertEquals(1000, Integer.parseInt(partialDoc.getFieldValue("price").toString()));
@@ -85,10 +87,10 @@ public class UpdateLogTest extends SolrTestCaseJ4 {
     if (random().nextBoolean()) { // sometimes also try a second commit
       ulogCommit(ulog);
     }
-    ulog.add(getAddUpdate(102L, sdoc("id", "1", "price", "2000", "val1_i_dvo", "4", "_version_", "200")));    
-    ulog.add(getAddUpdate(200L, sdoc("id", "1", "val1_i_dvo", "5", "_version_", "201")));
+    ulogAdd(ulog, 102L, sdoc("id", "1", "price", "2000", "val1_i_dvo", "4", "_version_", "200"));
+    ulogAdd(ulog, 200L, sdoc("id", "1", "val1_i_dvo", "5", "_version_", "201"));
 
-    partialUpdate = ulog.lookup(cmd.getIndexedId());
+    partialUpdate = ulog.lookup(DOC_1_INDEXED_ID);
     partialDoc = RealTimeGetComponent.toSolrDoc((SolrInputDocument)((List)partialUpdate).get(4), h.getCore().getLatestSchema(), true);
     prevVersion = (Long)((List)partialUpdate).get(3);
     prevPointer = (Long)((List)partialUpdate).get(2);
@@ -96,7 +98,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, null, partialDoc);
+    returnVal = ulog.applyPartialUpdates(DOC_1_INDEXED_ID, prevPointer, prevVersion, null, partialDoc);
 
     assertEquals(0, returnVal);
     assertEquals(2000, Integer.parseInt(partialDoc.getFieldValue("price").toString()));
@@ -106,18 +108,16 @@ public class UpdateLogTest extends SolrTestCaseJ4 {
   
   @Test
   public void testApplyPartialUpdatesAfterMultipleCommits() {    
-    AddUpdateCommand cmd;
-    cmd = getAddUpdate(null, sdoc("id", "1", "title_s", "title1", "val1_i_dvo", "1", "_version_", "100"));
-    ulog.add(cmd);
-    ulog.add(getAddUpdate(100L, sdoc("id", "1", "price", "1000", "val1_i_dvo", "2", "_version_", "101")));
-    ulog.add(getAddUpdate(101L, sdoc("id", "1", "val1_i_dvo", "3", "_version_", "102")));
+    ulogAdd(ulog, null, sdoc("id", "1", "title_s", "title1", "val1_i_dvo", "1", "_version_", "100"));
+    ulogAdd(ulog, 100L, sdoc("id", "1", "price", "1000", "val1_i_dvo", "2", "_version_", "101"));
+    ulogAdd(ulog, 101L, sdoc("id", "1", "val1_i_dvo", "3", "_version_", "102"));
 
     // Do 3 commits, then in-place update, and verify that applying partial updates can't find full doc
     for (int i=0; i<3; i++)
       ulogCommit(ulog);
-    ulog.add(getAddUpdate(101L, sdoc("id", "1", "val1_i_dvo", "6", "_version_", "300")));
+    ulogAdd(ulog, 101L, sdoc("id", "1", "val1_i_dvo", "6", "_version_", "300"));
 
-    Object partialUpdate = ulog.lookup(cmd.getIndexedId());
+    Object partialUpdate = ulog.lookup(DOC_1_INDEXED_ID);
     SolrDocument partialDoc = RealTimeGetComponent.toSolrDoc((SolrInputDocument)((List)partialUpdate).get(4), h.getCore().getLatestSchema(), true);
     long prevVersion = (Long)((List)partialUpdate).get(3);
     long prevPointer = (Long)((List)partialUpdate).get(2);
@@ -125,22 +125,19 @@ 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, null, partialDoc);
+    long returnVal = ulog.applyPartialUpdates(DOC_1_INDEXED_ID, prevPointer, prevVersion, null, partialDoc);
 
     assertEquals(-1, returnVal);
   }
 
   @Test
   public void testApplyPartialUpdatesDependingOnNonAddShouldThrowException() {
-    AddUpdateCommand cmd;
-    cmd = getAddUpdate(null, sdoc("id", "1", "title_s", "title1", "val1_i_dvo", "1", "_version_", "100"));
-    ulog.add(cmd);
+    ulogAdd(ulog, null, sdoc("id", "1", "title_s", "title1", "val1_i_dvo", "1", "_version_", "100"));
+    ulogDelete(ulog, "1", 500L, false); // dbi
+    ulogAdd(ulog, 500L, sdoc("id", "1", "val1_i_dvo", "2", "_version_", "501"));
+    ulogAdd(ulog, 501L, sdoc("id", "1", "val1_i_dvo", "3", "_version_", "502"));
 
-    ulog.delete(getDeleteUpdate("1", 500L, false)); // dbi
-    ulog.add(getAddUpdate(500L, sdoc("id", "1", "val1_i_dvo", "2", "_version_", "501")));
-    ulog.add(getAddUpdate(501L, sdoc("id", "1", "val1_i_dvo", "3", "_version_", "502")));
-
-    Object partialUpdate = ulog.lookup(cmd.getIndexedId());
+    Object partialUpdate = ulog.lookup(DOC_1_INDEXED_ID);
     SolrDocument partialDoc = RealTimeGetComponent.toSolrDoc((SolrInputDocument)((List)partialUpdate).get(4), h.getCore().getLatestSchema(), true);
     long prevVersion = (Long)((List)partialUpdate).get(3);
     long prevPointer = (Long)((List)partialUpdate).get(2);
@@ -151,7 +148,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, null, partialDoc);
+        long returnVal = ulog.applyPartialUpdates(DOC_1_INDEXED_ID, 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);
       });
@@ -164,23 +161,21 @@ public class UpdateLogTest extends SolrTestCaseJ4 {
   public void testApplyPartialUpdatesWithDBQ() { // nocommit: missleading name?
 
     // nocommit: no in-place updates happening in this test?
-    
-    AddUpdateCommand cmd;
-    cmd = getAddUpdate(null, sdoc("id", "1", "title_s", "title1", "val1_i_dvo", "1", "_version_", "100"));
-    ulog.add(cmd);
-    ulog.add(getAddUpdate(100L, sdoc("id", "1", "val1_i_dvo", "2", "_version_", "101")));
-    ulog.add(getAddUpdate(101L, sdoc("id", "1", "val1_i_dvo", "3", "_version_", "102")));
-    ulog.deleteByQuery(getDeleteUpdate("1", 200L, true)); // dbq, "id:1"
-    assertNull(ulog.lookup(cmd.getIndexedId()));
+ 
+    ulogAdd(ulog, null, sdoc("id", "1", "title_s", "title1", "val1_i_dvo", "1", "_version_", "100"));
+    ulogAdd(ulog, 100L, sdoc("id", "1", "val1_i_dvo", "2", "_version_", "101"));
+    ulogAdd(ulog, 101L, sdoc("id", "1", "val1_i_dvo", "3", "_version_", "102"));
+    ulogDelete(ulog, "1", 200L, true); // dbq, "id:1"
+    assertNull(ulog.lookup(DOC_1_INDEXED_ID));
 
     // nocommit: need more rigerous assertions about expected behavior after DBQ (new RT searcher?)
   }
 
   /**
-   * Simulate a commit at a given updateLog
+   * Simulate a commit on a given updateLog
    */
   private static void ulogCommit(UpdateLog ulog) {
-    try (SolrQueryRequest req = new LocalSolrQueryRequest(h.getCore(), params())) {
+    try (SolrQueryRequest req = req()) {
       CommitUpdateCommand commitCmd = new CommitUpdateCommand(req, false);
       ulog.preCommit(commitCmd);
       ulog.postCommit(commitCmd);
@@ -188,46 +183,64 @@ public class UpdateLogTest extends SolrTestCaseJ4 {
   }
 
   /**
-   * Obtain a DeleteUpdateCommand for a deleteById operation
+   * Simulate a delete on a given updateLog
+   *
+   * @param ulog The UpdateLog to apply a delete against
+   * @param id of document to be deleted
+   * @param version Version to use on the DeleteUpdateCommand
+   * @param dbq if true, an <code>id:$id</code> DBQ will used, instead of delete by id
    */
-  private static DeleteUpdateCommand getDeleteUpdate(String id, long version, boolean dbq) {
-    // nocommit: req lifecycle bug
-    // nocommit: cmd returned is linked to req that's already been closed
-    // nocommit: see jira comments for suggested fix
-    try (SolrQueryRequest req = new LocalSolrQueryRequest(h.getCore(), params())) {
+  private static void ulogDelete(UpdateLog ulog, String id, long version, boolean dbq) {
+    try (SolrQueryRequest req = req()) {
       DeleteUpdateCommand cmd = new DeleteUpdateCommand(req);
+      cmd.setVersion(version);
       if (dbq) {
         cmd.query = ("id:"+id);
+        ulog.deleteByQuery(cmd);
       } else {
         cmd.id = id;
+        ulog.delete(cmd);
       }
-      cmd.setVersion(version);
-      return cmd;
     }
   }
 
   /**
-   *   Obtain an AddUpdateCommand for a full add/in-place update operation.
-   *   If there's a non-null prevVersion, then this AddUpdateCommand represents an in-place update.
+   * Simulate an add on a given updateLog.
+   * <p>
    *   This method, when prevVersion is passed in (i.e. for in-place update), represents an 
    *   AddUpdateCommand that has undergone the merge process and inc/set operations have now been
    *   converted into actual values that just need to be written. 
+   * </p>
+   * <p>
+   * NOTE: For test simplicity, the Solr input document must include the <code>_version_</code> field.
+   * </p>
+   *
+   * @param ulog The UpdateLog to apply a delete against
+   * @param prevVersion If non-null, then this AddUpdateCommand represents an in-place update.
+   * @param sdoc The document to use for the add.
+   * @see #buildAddUpdateCommand
    */
-  public static AddUpdateCommand getAddUpdate(Long prevVersion, SolrInputDocument sdoc) {
-    // nocommit: req lifecycle bug
-    // nocommit: cmd returned is linked to req that's already been closed
-    // nocommit: see jira comments for suggested fix
-    AddUpdateCommand cmd; 
-    try (SolrQueryRequest req = new LocalSolrQueryRequest(h.getCore(), params())) {
-      cmd = new AddUpdateCommand(req);
+  private static void ulogAdd(UpdateLog ulog, Long prevVersion, SolrInputDocument sdoc) {
+    try (SolrQueryRequest req = req()) {
+      AddUpdateCommand cmd = buildAddUpdateCommand(req, sdoc);
+      if (prevVersion != null) {
+        cmd.prevVersion = prevVersion;
+      }
+      ulog.add(cmd);
     }
+  }
+
+  /**
+   * Helper method to construct an <code>AddUpdateCommand</code> for a <code>SolrInputDocument</code> 
+   * in the context of the specified <code>SolrQueryRequest</code>. 
+   *
+   * NOTE: For test simplicity, the Solr input document must include the <code>_version_</code> field.
+   */ 
+  public static AddUpdateCommand buildAddUpdateCommand(final SolrQueryRequest req, final SolrInputDocument sdoc) {
+    AddUpdateCommand cmd = new AddUpdateCommand(req);
     cmd.solrDoc = sdoc;
-    assertTrue(cmd.solrDoc.containsKey(DistributedUpdateProcessor.VERSION_FIELD));
+    assertTrue("", cmd.solrDoc.containsKey(DistributedUpdateProcessor.VERSION_FIELD));
     cmd.setVersion(Long.parseLong(cmd.solrDoc.getFieldValue(DistributedUpdateProcessor.VERSION_FIELD).toString()));
-    if (prevVersion != null) {
-      cmd.prevVersion = prevVersion;
-    }
     return cmd;
   }
-
 }