You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by no...@apache.org on 2017/01/26 07:34:33 UTC

[08/12] lucene-solr:apiv2: SOLR-5944: In-place updates of Numeric DocValues

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/53754108/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
new file mode 100644
index 0000000..9a5031f
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java
@@ -0,0 +1,1100 @@
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.update;
+
+import static org.junit.internal.matchers.StringContains.containsString;
+import static org.apache.solr.update.UpdateLogTest.buildAddUpdateCommand;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer;
+import org.apache.solr.common.SolrDocument;
+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.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;
+import org.apache.solr.update.processor.AtomicUpdateDocumentMerger;
+import org.apache.solr.util.RefCounted;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+
+/**
+ * Tests the in-place updates (docValues updates) for a standalone Solr instance.
+ */
+public class TestInPlaceUpdatesStandalone extends SolrTestCaseJ4 {
+  private static SolrClient client;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    System.setProperty("solr.tests.intClassName", random().nextBoolean()? "TrieIntField": "IntPointField");
+    System.setProperty("solr.tests.longClassName", random().nextBoolean()? "TrieLongField": "LongPointField");
+    System.setProperty("solr.tests.floatClassName", random().nextBoolean()? "TrieFloatField": "FloatPointField");
+    System.setProperty("solr.tests.doubleClassName", random().nextBoolean()? "TrieDoubleField": "DoublePointField");
+
+    initCore("solrconfig-tlog.xml", "schema-inplace-updates.xml");
+
+    // sanity check that autocommits are disabled
+    assertEquals(-1, h.getCore().getSolrConfig().getUpdateHandlerInfo().autoCommmitMaxTime);
+    assertEquals(-1, h.getCore().getSolrConfig().getUpdateHandlerInfo().autoSoftCommmitMaxTime);
+    assertEquals(-1, h.getCore().getSolrConfig().getUpdateHandlerInfo().autoCommmitMaxDocs);
+    assertEquals(-1, h.getCore().getSolrConfig().getUpdateHandlerInfo().autoSoftCommmitMaxDocs);
+
+    // validate that the schema was not changed to an unexpected state
+    IndexSchema schema = h.getCore().getLatestSchema();
+    for (String fieldName : Arrays.asList("_version_",
+                                          "inplace_l_dvo",
+                                          "inplace_updatable_float",
+                                          "inplace_updatable_int", 
+                                          "inplace_updatable_float_with_default",
+                                          "inplace_updatable_int_with_default")) {
+      // these fields must only be using docValues to support inplace updates
+      SchemaField field = schema.getField(fieldName);
+      assertTrue(field.toString(),
+                 field.hasDocValues() && ! field.indexed() && ! field.stored());
+    }
+    for (String fieldName : Arrays.asList("title_s", "regular_l", "stored_i")) {
+      // these fields must support atomic updates, but not inplace updates (ie: stored)
+      SchemaField field = schema.getField(fieldName);
+      assertTrue(field.toString(), field.stored());
+    }    
+
+    // Don't close this client, it would shutdown the CoreContainer
+    client = new EmbeddedSolrServer(h.getCoreContainer(), h.coreName);
+  }
+
+  @After
+  public void after() {
+    System.clearProperty("solr.tests.intClassName");
+    System.clearProperty("solr.tests.longClassName");
+    System.clearProperty("solr.tests.floatClassName");
+    System.clearProperty("solr.tests.doubleClassName");
+  }
+
+  @Before
+  public void deleteAllAndCommit() throws Exception {
+    clearIndex();
+    assertU(commit("softCommit", "false"));
+  }
+
+  @Test
+  public void testUpdatingDocValues() throws Exception {
+    long version1 = addAndGetVersion(sdoc("id", "1", "title_s", "first", "inplace_updatable_float", 41), null);
+    long version2 = addAndGetVersion(sdoc("id", "2", "title_s", "second", "inplace_updatable_float", 42), null);
+    long version3 = addAndGetVersion(sdoc("id", "3", "title_s", "third", "inplace_updatable_float", 43), null);
+    assertU(commit("softCommit", "false"));
+    assertQ(req("q", "*:*"), "//*[@numFound='3']");
+
+    // the reason we're fetching these docids is to validate that the subsequent updates 
+    // are done in place and don't cause the docids to change
+    int docid1 = getDocId("1");
+    int docid2 = getDocId("2");
+    int docid3 = getDocId("3");
+
+    // Check docValues were "set"
+    version1 = addAndAssertVersion(version1, "id", "1", "inplace_updatable_float", map("set", 200));
+    version2 = addAndAssertVersion(version2, "id", "2", "inplace_updatable_float", map("set", 300));
+    version3 = addAndAssertVersion(version3, "id", "3", "inplace_updatable_float", map("set", 100));
+    assertU(commit("softCommit", "false"));
+
+    assertQ(req("q", "*:*", "sort", "id asc", "fl", "*,[docid]"),
+        "//*[@numFound='3']",
+        "//result/doc[1]/float[@name='inplace_updatable_float'][.='200.0']",
+        "//result/doc[2]/float[@name='inplace_updatable_float'][.='300.0']",
+        "//result/doc[3]/float[@name='inplace_updatable_float'][.='100.0']",
+        "//result/doc[1]/long[@name='_version_'][.='"+version1+"']",
+        "//result/doc[2]/long[@name='_version_'][.='"+version2+"']",
+        "//result/doc[3]/long[@name='_version_'][.='"+version3+"']",
+        "//result/doc[1]/int[@name='[docid]'][.='"+docid1+"']",
+        "//result/doc[2]/int[@name='[docid]'][.='"+docid2+"']",
+        "//result/doc[3]/int[@name='[docid]'][.='"+docid3+"']"
+        );
+
+    // Check docValues are "inc"ed
+    version1 = addAndAssertVersion(version1, "id", "1", "inplace_updatable_float", map("inc", 1));
+    version2 = addAndAssertVersion(version2, "id", "2", "inplace_updatable_float", map("inc", -2));
+    version3 = addAndAssertVersion(version3, "id", "3", "inplace_updatable_float", map("inc", 3));
+    assertU(commit("softCommit", "false"));
+    assertQ(req("q", "*:*", "sort", "id asc", "fl", "*,[docid]"),
+        "//*[@numFound='3']",
+        "//result/doc[1]/float[@name='inplace_updatable_float'][.='201.0']",
+        "//result/doc[2]/float[@name='inplace_updatable_float'][.='298.0']",
+        "//result/doc[3]/float[@name='inplace_updatable_float'][.='103.0']",
+        "//result/doc[1]/long[@name='_version_'][.='"+version1+"']",
+        "//result/doc[2]/long[@name='_version_'][.='"+version2+"']",
+        "//result/doc[3]/long[@name='_version_'][.='"+version3+"']",
+        "//result/doc[1]/int[@name='[docid]'][.='"+docid1+"']",
+        "//result/doc[2]/int[@name='[docid]'][.='"+docid2+"']",
+        "//result/doc[3]/int[@name='[docid]'][.='"+docid3+"']"
+        );
+
+    // Check back to back "inc"s are working (off the transaction log)
+    version1 = addAndAssertVersion(version1, "id", "1", "inplace_updatable_float", map("inc", 1));
+    version1 = addAndAssertVersion(version1, "id", "1", "inplace_updatable_float", map("inc", 2)); // new value should be 204
+    assertU(commit("softCommit", "false"));
+    assertQ(req("q", "id:1", "fl", "*,[docid]"),
+        "//result/doc[1]/float[@name='inplace_updatable_float'][.='204.0']",
+        "//result/doc[1]/long[@name='_version_'][.='"+version1+"']",
+        "//result/doc[1]/int[@name='[docid]'][.='"+docid1+"']");
+
+    // Now let the document be atomically updated (non-inplace), ensure the old docvalue is part of new doc
+    version1 = addAndAssertVersion(version1, "id", "1", "title_s", map("set", "new first"));
+    assertU(commit("softCommit", "false"));
+    int newDocid1 = getDocId("1");
+    assertTrue(newDocid1 != docid1);
+    docid1 = newDocid1;
+
+    assertQ(req("q", "id:1"),
+        "//result/doc[1]/float[@name='inplace_updatable_float'][.='204.0']",
+        "//result/doc[1]/str[@name='title_s'][.='new first']",
+        "//result/doc[1]/long[@name='_version_'][.='"+version1+"']");
+
+    // Check if atomic update with "inc" to a docValue works
+    version2 = addAndAssertVersion(version2, "id", "2", "title_s", map("set", "new second"), "inplace_updatable_float", map("inc", 2));
+    assertU(commit("softCommit", "false"));
+    int newDocid2 = getDocId("2");
+    assertTrue(newDocid2 != docid2);
+    docid2 = newDocid2;
+
+    assertQ(req("q", "id:2"),
+        "//result/doc[1]/float[@name='inplace_updatable_float'][.='300.0']",
+        "//result/doc[1]/str[@name='title_s'][.='new second']",
+        "//result/doc[1]/long[@name='_version_'][.='"+version2+"']");
+
+    // Check if docvalue "inc" update works for a newly created document, which is not yet committed
+    // Case1: docvalue was supplied during add of new document
+    long version4 = addAndGetVersion(sdoc("id", "4", "title_s", "fourth", "inplace_updatable_float", "400"), params());
+    version4 = addAndAssertVersion(version4, "id", "4", "inplace_updatable_float", map("inc", 1));
+    assertU(commit("softCommit", "false"));
+    assertQ(req("q", "id:4"),
+        "//result/doc[1]/float[@name='inplace_updatable_float'][.='401.0']",
+        "//result/doc[1]/long[@name='_version_'][.='"+version4+"']");
+
+    // Check if docvalue "inc" update works for a newly created document, which is not yet committed
+    // Case2: docvalue was not supplied during add of new document, should assume default
+    long version5 = addAndGetVersion(sdoc("id", "5", "title_s", "fifth"), params());
+    version5 = addAndAssertVersion(version5, "id", "5", "inplace_updatable_float", map("inc", 1));
+    assertU(commit("softCommit", "false"));
+    assertQ(req("q", "id:5"),
+        "//result/doc[1]/float[@name='inplace_updatable_float'][.='1.0']",
+        "//result/doc[1]/long[@name='_version_'][.='"+version5+"']");
+
+    // Check if docvalue "set" update works for a newly created document, which is not yet committed
+    long version6 = addAndGetVersion(sdoc("id", "6", "title_s", "sixth"), params());
+    version6 = addAndAssertVersion(version6, "id", "6", "inplace_updatable_float", map("set", 600));
+    assertU(commit("softCommit", "false"));
+    assertQ(req("q", "id:6"),
+        "//result/doc[1]/float[@name='inplace_updatable_float'][.='600.0']",
+        "//result/doc[1]/long[@name='_version_'][.='"+version6+"']");
+
+    // Check optimistic concurrency works
+    long v20 = addAndGetVersion(sdoc("id", "20", "title_s","first", "inplace_updatable_float", 100), params());    
+    SolrException exception = expectThrows(SolrException.class, () -> {
+      addAndGetVersion(sdoc("id","20", "_version_", -1, "inplace_updatable_float", map("inc", 1)), null);
+    });
+    assertEquals(exception.toString(), SolrException.ErrorCode.CONFLICT.code, exception.code());
+    assertThat(exception.getMessage(), containsString("expected=-1"));
+    assertThat(exception.getMessage(), containsString("actual="+v20));
+
+
+    long oldV20 = v20;
+    v20 = addAndAssertVersion(v20, "id","20", "_version_", v20, "inplace_updatable_float", map("inc", 1));
+    exception = expectThrows(SolrException.class, () -> {
+      addAndGetVersion(sdoc("id","20", "_version_", oldV20, "inplace_updatable_float", map("inc", 1)), null);
+    });
+    assertEquals(exception.toString(), SolrException.ErrorCode.CONFLICT.code, exception.code());
+    assertThat(exception.getMessage(), containsString("expected="+oldV20));
+    assertThat(exception.getMessage(), containsString("actual="+v20));
+
+    v20 = addAndAssertVersion(v20, "id","20", "_version_", v20, "inplace_updatable_float", map("inc", 1));
+    // RTG before a commit
+    assertJQ(req("qt","/get", "id","20", "fl","id,inplace_updatable_float,_version_"),
+        "=={'doc':{'id':'20', 'inplace_updatable_float':" + 102.0 + ",'_version_':" + v20 + "}}");
+    assertU(commit("softCommit", "false"));
+    assertQ(req("q", "id:20"), 
+        "//result/doc[1]/float[@name='inplace_updatable_float'][.='102.0']",
+        "//result/doc[1]/long[@name='_version_'][.='"+v20+"']");
+
+    // Check if updated DVs can be used for search
+    assertQ(req("q", "inplace_updatable_float:102"), 
+        "//result/doc[1]/str[@name='id'][.='20']",
+        "//result/doc[1]/float[@name='inplace_updatable_float'][.='102.0']",
+        "//result/doc[1]/long[@name='_version_'][.='"+v20+"']");
+
+    // Check if updated DVs can be used for sorting
+    assertQ(req("q", "*:*", "sort", "inplace_updatable_float asc"), 
+        "//result/doc[4]/str[@name='id'][.='1']",
+        "//result/doc[4]/float[@name='inplace_updatable_float'][.='204.0']",
+
+        "//result/doc[5]/str[@name='id'][.='2']",
+        "//result/doc[5]/float[@name='inplace_updatable_float'][.='300.0']",
+
+        "//result/doc[3]/str[@name='id'][.='3']",
+        "//result/doc[3]/float[@name='inplace_updatable_float'][.='103.0']",
+
+        "//result/doc[6]/str[@name='id'][.='4']",
+        "//result/doc[6]/float[@name='inplace_updatable_float'][.='401.0']",
+
+        "//result/doc[1]/str[@name='id'][.='5']",
+        "//result/doc[1]/float[@name='inplace_updatable_float'][.='1.0']",
+
+        "//result/doc[7]/str[@name='id'][.='6']",
+        "//result/doc[7]/float[@name='inplace_updatable_float'][.='600.0']",
+
+        "//result/doc[2]/str[@name='id'][.='20']",
+        "//result/doc[2]/float[@name='inplace_updatable_float'][.='102.0']");
+  }
+
+  @Test
+  public void testUpdateTwoDifferentFields() throws Exception {
+    long version1 = addAndGetVersion(sdoc("id", "1", "title_s", "first", "inplace_updatable_float", 42), null);
+    assertU(commit("softCommit", "false"));
+    assertQ(req("q", "*:*"), "//*[@numFound='1']");
+
+    int docid1 = getDocId("1");
+
+    // Check docValues were "set"
+    version1 = addAndAssertVersion(version1, "id", "1", "inplace_updatable_float", map("set", 200));
+    version1 = addAndAssertVersion(version1, "id", "1", "inplace_updatable_int", map("set", 10));
+    assertU(commit("softCommit", "false"));
+
+    assertU(commit("softCommit", "false"));
+
+    assertQ(req("q", "*:*", "sort", "id asc", "fl", "*,[docid]"),
+        "//*[@numFound='1']",
+        "//result/doc[1]/float[@name='inplace_updatable_float'][.='200.0']",
+        "//result/doc[1]/long[@name='_version_'][.='"+version1+"']",
+        "//result/doc[1]/int[@name='[docid]'][.='"+docid1+"']"
+        );
+
+    // two different update commands, updating each of the fields separately
+    version1 = addAndAssertVersion(version1, "id", "1", "inplace_updatable_int", map("inc", 1));
+    version1 = addAndAssertVersion(version1, "id", "1", "inplace_updatable_float", map("inc", 1));
+    // same update command, updating both the fields together
+    version1 = addAndAssertVersion(version1, "id", "1", "inplace_updatable_int", map("inc", 1),
+        "inplace_updatable_float", map("inc", 1));
+
+    if (random().nextBoolean()) {
+      assertU(commit("softCommit", "false"));
+      assertQ(req("q", "*:*", "sort", "id asc", "fl", "*,[docid]"),
+          "//*[@numFound='1']",
+          "//result/doc[1]/float[@name='inplace_updatable_float'][.='202.0']",
+          "//result/doc[1]/int[@name='inplace_updatable_int'][.='12']",
+          "//result/doc[1]/long[@name='_version_'][.='"+version1+"']",
+          "//result/doc[1]/int[@name='[docid]'][.='"+docid1+"']"
+          );
+    } 
+
+    // RTG
+    assertJQ(req("qt","/get", "id","1", "fl","id,inplace_updatable_float,inplace_updatable_int"),
+        "=={'doc':{'id':'1', 'inplace_updatable_float':" + 202.0 + ",'inplace_updatable_int':" + 12 + "}}");
+
+  }
+
+  @Test
+  public void testDVUpdatesWithDBQofUpdatedValue() throws Exception {
+    long version1 = addAndGetVersion(sdoc("id", "1", "title_s", "first", "inplace_updatable_float", "0"), null);
+    assertU(commit());
+
+    // in-place update
+    addAndAssertVersion(version1, "id", "1", "inplace_updatable_float", map("set", 100), "_version_", version1);
+
+    // DBQ where q=inplace_updatable_float:100
+    assertU(delQ("inplace_updatable_float:100"));
+
+    assertU(commit());
+
+    assertQ(req("q", "*:*"), "//*[@numFound='0']");
+  }
+
+  @Test
+  public void testDVUpdatesWithDelete() throws Exception {
+    long version1 = 0;
+
+    for (boolean postAddCommit : Arrays.asList(true, false)) {
+      for (boolean delById : Arrays.asList(true, false)) {
+        for (boolean postDelCommit : Arrays.asList(true, false)) {
+          addAndGetVersion(sdoc("id", "1", "title_s", "first"), params());
+          if (postAddCommit) assertU(commit());
+          assertU(delById ? delI("1") : delQ("id:1"));
+          if (postDelCommit) assertU(commit());
+          version1 = addAndGetVersion(sdoc("id", "1", "inplace_updatable_float", map("set", 200)), params());
+          // assert current doc#1 doesn't have old value of "title_s"
+          assertU(commit());
+          assertQ(req("q", "title_s:first", "sort", "id asc", "fl", "*,[docid]"),
+              "//*[@numFound='0']");
+        }
+      }
+    }
+
+    // Update to recently deleted (or non-existent) document with a "set" on updatable 
+    // field should succeed, since it is executed internally as a full update
+    // because AUDM.doInPlaceUpdateMerge() returns false
+    assertU(random().nextBoolean()? delI("1"): delQ("id:1"));
+    if (random().nextBoolean()) assertU(commit());
+    addAndAssertVersion(version1, "id", "1", "inplace_updatable_float", map("set", 200));
+    assertU(commit());
+    assertQ(req("q", "id:1", "sort", "id asc", "fl", "*"),
+        "//*[@numFound='1']",
+        "//result/doc[1]/float[@name='inplace_updatable_float'][.='200.0']");
+
+    // Another "set" on the same field should be an in-place update 
+    int docid1 = getDocId("1");
+    addAndAssertVersion(version1, "id", "1", "inplace_updatable_float", map("set", 300));
+    assertU(commit());
+    assertQ(req("q", "id:1", "fl", "*,[docid]"),
+        "//result/doc[1]/float[@name='inplace_updatable_float'][.='300.0']",
+        "//result/doc[1]/int[@name='[docid]'][.='"+docid1+"']");
+  }
+
+  public static long addAndAssertVersion(long expectedCurrentVersion, Object... fields) throws Exception {
+    assert 0 < expectedCurrentVersion;
+    long currentVersion = addAndGetVersion(sdoc(fields), null);
+    assertTrue(currentVersion > expectedCurrentVersion);
+    return currentVersion;
+  }
+
+  /**
+   * Helper method to search for the specified (uniqueKey field) id using <code>fl=[docid]</code> 
+   * and return the internal lucene docid.
+   */
+  private int getDocId(String id) throws NumberFormatException, Exception {
+    SolrDocumentList results = client.query(params("q","id:" + id, "fl", "[docid]")).getResults();
+    assertEquals(1, results.getNumFound());
+    assertEquals(1, results.size());
+    Object docid = results.get(0).getFieldValue("[docid]");
+    assertTrue(docid instanceof Integer);
+    return ((Integer)docid);
+  }
+
+  @Test
+  public void testUpdateOfNonExistentDVsShouldNotFail() throws Exception {
+    // schema sanity check: assert that the nonexistent_field_i_dvo doesn't exist already
+    FieldInfo fi;
+    RefCounted<SolrIndexSearcher> holder = h.getCore().getSearcher();
+    try {
+      fi = holder.get().getSlowAtomicReader().getFieldInfos().fieldInfo("nonexistent_field_i_dvo");
+    } finally {
+      holder.decref();
+    }
+    assertNull(fi);
+
+    // Partial update
+    addAndGetVersion(sdoc("id", "0", "nonexistent_field_i_dvo", map("set", "42")), null);
+
+    addAndGetVersion(sdoc("id", "1"), null);
+    addAndGetVersion(sdoc("id", "1", "nonexistent_field_i_dvo", map("inc", "1")), null);
+    addAndGetVersion(sdoc("id", "1", "nonexistent_field_i_dvo", map("inc", "1")), null);
+
+    assertU(commit());
+
+    assertQ(req("q", "*:*"), "//*[@numFound='2']");    
+    assertQ(req("q", "nonexistent_field_i_dvo:42"), "//*[@numFound='1']");    
+    assertQ(req("q", "nonexistent_field_i_dvo:2"), "//*[@numFound='1']");    
+  }
+
+  @Test
+  public void testOnlyPartialUpdatesBetweenCommits() throws Exception {
+    // Full updates
+    long version1 = addAndGetVersion(sdoc("id", "1", "title_s", "first", "val1_i_dvo", "1", "val2_l_dvo", "1"), params());
+    long version2 = addAndGetVersion(sdoc("id", "2", "title_s", "second", "val1_i_dvo", "2", "val2_l_dvo", "2"), params());
+    long version3 = addAndGetVersion(sdoc("id", "3", "title_s", "third", "val1_i_dvo", "3", "val2_l_dvo", "3"), params());
+    assertU(commit("softCommit", "false"));
+
+    assertQ(req("q", "*:*", "fl", "*,[docid]"), "//*[@numFound='3']");
+
+    int docid1 = getDocId("1");
+    int docid2 = getDocId("2");
+    int docid3 = getDocId("3");
+
+    int numPartialUpdates = 1 + random().nextInt(5000);
+    for (int i=0; i<numPartialUpdates; i++) {
+      version1 = addAndAssertVersion(version1, "id", "1", "val1_i_dvo", map("set", i));
+      version2 = addAndAssertVersion(version2, "id", "2", "val1_i_dvo", map("inc", 1));
+      version3 = addAndAssertVersion(version3, "id", "3", "val1_i_dvo", map("set", i));
+
+      version1 = addAndAssertVersion(version1, "id", "1", "val2_l_dvo", map("set", i));
+      version2 = addAndAssertVersion(version2, "id", "2", "val2_l_dvo", map("inc", 1));
+      version3 = addAndAssertVersion(version3, "id", "3", "val2_l_dvo", map("set", i));
+    }
+    assertU(commit("softCommit", "true"));
+
+    assertQ(req("q", "*:*", "sort", "id asc", "fl", "*,[docid]"),
+        "//*[@numFound='3']",
+        "//result/doc[1]/int[@name='val1_i_dvo'][.='"+(numPartialUpdates-1)+"']",
+        "//result/doc[2]/int[@name='val1_i_dvo'][.='"+(numPartialUpdates+2)+"']",
+        "//result/doc[3]/int[@name='val1_i_dvo'][.='"+(numPartialUpdates-1)+"']",
+        "//result/doc[1]/long[@name='val2_l_dvo'][.='"+(numPartialUpdates-1)+"']",
+        "//result/doc[2]/long[@name='val2_l_dvo'][.='"+(numPartialUpdates+2)+"']",
+        "//result/doc[3]/long[@name='val2_l_dvo'][.='"+(numPartialUpdates-1)+"']",
+        "//result/doc[1]/int[@name='[docid]'][.='"+docid1+"']",
+        "//result/doc[2]/int[@name='[docid]'][.='"+docid2+"']",
+        "//result/doc[3]/int[@name='[docid]'][.='"+docid3+"']",
+        "//result/doc[1]/long[@name='_version_'][.='" + version1 + "']",
+        "//result/doc[2]/long[@name='_version_'][.='" + version2 + "']",
+        "//result/doc[3]/long[@name='_version_'][.='" + version3 + "']"
+        );
+  }
+
+  /**
+   * Useful to store the state of an expected document into an in-memory model
+   * representing the index.
+   */
+  private static class DocInfo {
+    public final long version;
+    public final Long value;
+
+    public DocInfo(long version, Long val) {
+      this.version = version;
+      this.value = val;
+    }
+
+    @Override
+    public String toString() {
+      return "["+version+", "+value+"]";
+    }
+  }
+
+  /** @see #checkReplay */
+  @Test
+  public void testReplay_AfterInitialAddMixOfIncAndSet() throws Exception {
+    checkReplay("val2_l_dvo",
+        //
+        sdoc("id", "0", "val2_l_dvo", 3000000000L),
+        sdoc("id", "0", "val2_l_dvo", map("inc", 3)),
+        HARDCOMMIT,
+        sdoc("id", "0", "val2_l_dvo", map("inc", 5)),
+        sdoc("id", "1", "val2_l_dvo", 2000000000L),
+        sdoc("id", "1", "val2_l_dvo", map("set", 2000000002L)),
+        sdoc("id", "1", "val2_l_dvo", map("set", 3000000000L)),
+        sdoc("id", "0", "val2_l_dvo", map("inc", 7)),
+        sdoc("id", "1", "val2_l_dvo", map("set", 7000000000L)),
+        sdoc("id", "0", "val2_l_dvo", map("inc", 11)),
+        sdoc("id", "2", "val2_l_dvo", 2000000000L),
+        HARDCOMMIT,
+        sdoc("id", "2", "val2_l_dvo", map("set", 3000000000L)),
+        HARDCOMMIT);
+  }
+
+  /** @see #checkReplay */
+  @Test
+  public void testReplay_AfterInitialAddMixOfIncAndSetAndFullUpdates() throws Exception {
+    checkReplay("val2_l_dvo",
+        //
+        sdoc("id", "0", "val2_l_dvo", 3000000000L),
+        sdoc("id", "0", "val2_l_dvo", map("set", 3000000003L)),
+        HARDCOMMIT,
+        sdoc("id", "0", "val2_l_dvo", map("set", 3000000008L)),
+        sdoc("id", "1", "val2_l_dvo", 2000000000L),
+        sdoc("id", "1", "val2_l_dvo", map("inc", 2)),
+        sdoc("id", "1", "val2_l_dvo", 3000000000L),
+        sdoc("id", "0", "val2_l_dvo", map("set", 3000000015L)),
+        sdoc("id", "1", "val2_l_dvo", 7000000000L),
+        sdoc("id", "0", "val2_l_dvo", map("set", 3000000026L)),
+        sdoc("id", "2", "val2_l_dvo", 2000000000L),
+        HARDCOMMIT,
+        sdoc("id", "2", "val2_l_dvo", 3000000000L),
+        HARDCOMMIT);
+  }
+
+  /** @see #checkReplay */
+  @Test
+  public void testReplay_AllUpdatesAfterInitialAddAreInc() throws Exception {
+    checkReplay("val2_l_dvo",
+        //
+        sdoc("id", "0", "val2_l_dvo", 3000000000L),
+        sdoc("id", "0", "val2_l_dvo", map("inc", 3)),
+        HARDCOMMIT,
+        sdoc("id", "0", "val2_l_dvo", map("inc", 5)),
+        sdoc("id", "1", "val2_l_dvo", 2000000000L),
+        sdoc("id", "1", "val2_l_dvo", map("inc", 2)),
+        sdoc("id", "1", "val2_l_dvo", 3000000000L),
+        sdoc("id", "0", "val2_l_dvo", map("inc", 7)),
+        sdoc("id", "1", "val2_l_dvo", 7000000000L),
+        sdoc("id", "0", "val2_l_dvo", map("inc", 11)),
+        sdoc("id", "2", "val2_l_dvo", 2000000000L),
+        HARDCOMMIT,
+        sdoc("id", "2", "val2_l_dvo", 3000000000L),
+        HARDCOMMIT);
+  }
+
+  /** @see #checkReplay */
+  @Test
+  public void testReplay_AllUpdatesAfterInitialAddAreSets() throws Exception {
+    checkReplay("val2_l_dvo",
+        //
+        sdoc("id", "0", "val2_l_dvo", 3000000000L),
+        sdoc("id", "0", "val2_l_dvo", map("set", 3000000003L)),
+        HARDCOMMIT,
+        sdoc("id", "0", "val2_l_dvo", map("set", 3000000008L)),
+        sdoc("id", "1", "val2_l_dvo", 2000000000L),
+        sdoc("id", "1", "val2_l_dvo", map("set", 2000000002L)),
+        sdoc("id", "1", "val2_l_dvo", map("set", 3000000000L)),
+        sdoc("id", "0", "val2_l_dvo", map("set", 3000000015L)),
+        sdoc("id", "1", "val2_l_dvo", map("set", 7000000000L)),
+        sdoc("id", "0", "val2_l_dvo", map("set", 3000000026L)),
+        sdoc("id", "2", "val2_l_dvo", 2000000000L),
+        HARDCOMMIT,
+        sdoc("id", "2", "val2_l_dvo", map("set", 3000000000L)),
+        HARDCOMMIT
+        );
+  }
+  
+  /** @see #checkReplay */
+  @Test
+  public void testReplay_MixOfInplaceAndNonInPlaceAtomicUpdates() throws Exception {
+    checkReplay("inplace_l_dvo",
+                //
+                sdoc("id", "3", "inplace_l_dvo", map("inc", -13)),
+                sdoc("id", "3", "inplace_l_dvo", map("inc", 19),    "regular_l", map("inc", -17)),
+                sdoc("id", "1",                                     "regular_l", map("inc", -19)),
+                sdoc("id", "3", "inplace_l_dvo", map("inc", -11)),
+                sdoc("id", "2", "inplace_l_dvo", map("set", 28)),
+                HARDCOMMIT,
+                sdoc("id", "2", "inplace_l_dvo", map("inc", 45)),
+                sdoc("id", "3", "inplace_l_dvo", map("set", 72)),
+                sdoc("id", "2",                                     "regular_l", map("inc", -55)),
+                sdoc("id", "2", "inplace_l_dvo", -48,               "regular_l", 159),
+                sdoc("id", "3", "inplace_l_dvo", 52,                "regular_l", 895),
+                sdoc("id", "2", "inplace_l_dvo", map("inc", 19)),
+                sdoc("id", "3", "inplace_l_dvo", map("inc", -264),  "regular_l", map("inc", -207)),
+                sdoc("id", "3", "inplace_l_dvo", -762,              "regular_l", 272),
+                SOFTCOMMIT);
+  }
+  
+  @Test
+  public void testReplay_SetOverriddenWithNoValueThenInc() throws Exception {
+    final String inplaceField = "inplace_l_dvo"; 
+    checkReplay(inplaceField,
+                //
+                sdoc("id", "1", inplaceField, map("set", 555L)),
+                SOFTCOMMIT,
+                sdoc("id", "1", "regular_l", 666L), // NOTE: no inplaceField, regular add w/overwrite 
+                sdoc("id", "1", inplaceField, map("inc", -77)),
+                HARDCOMMIT);
+  }
+
+  /** 
+   * Simple enum for randomizing a type of update.
+   * Each enum value has an associated probability, and the class has built in sanity checks 
+   * that the total is 100%
+   * 
+   * @see RandomUpdate#pick
+   * @see #checkRandomReplay
+   */
+  private static enum RandomUpdate {
+    HARD_COMMIT(5), 
+    SOFT_COMMIT(5),
+
+    /** doc w/o the inplaceField, atomic update on some other (non-inplace) field */
+    ATOMIC_NOT_INPLACE(5),
+    
+    /** atomic update of a doc w/ inc on both inplaceField *AND* non-inplace field */
+    ATOMIC_INPLACE_AND_NOT_INPLACE(10), 
+
+    
+    /** atomic update of a doc w/ set inplaceField */
+    ATOMIC_INPLACE_SET(25),
+    /** atomic update of a doc w/ inc inplaceField */
+    ATOMIC_INPLACE_INC(25), 
+    
+    /** doc w/o the inplaceField, normal add */
+    ADD_NO_INPLACE_VALUE(5),
+    /** a non atomic update of a doc w/ new inplaceField value */
+    ADD_INPLACE_VALUE(20); 
+    
+    private RandomUpdate(int odds) {
+      this.odds = odds;
+    }
+    public final int odds;
+
+    static { // sanity check odds add up to 100%
+      int total = 0;
+      for (RandomUpdate candidate : RandomUpdate.values()) {
+        total += candidate.odds;
+      }
+      assertEquals("total odds doesn't equal 100", 100, total);
+    }
+
+    /** pick a random type of RandomUpdate */
+    public static final RandomUpdate pick(Random r) {
+      final int target = TestUtil.nextInt(r, 1, 100);
+      int cumulative_odds = 0;
+      for (RandomUpdate candidate : RandomUpdate.values()) {
+        cumulative_odds += candidate.odds;
+        if (target <= cumulative_odds) {
+          return candidate;
+        }
+      }
+      fail("how did we not find a candidate? target=" + target + ", cumulative_odds=" + cumulative_odds);
+      return null; // compiler mandated return
+    }
+  }
+  
+  /** @see #checkRandomReplay */
+  @Test
+  public void testReplay_Random_ManyDocsManyUpdates() throws Exception {
+    
+    // build up a random list of updates
+    final int maxDocId = atLeast(50);
+    final int numUpdates = maxDocId * 3;
+    checkRandomReplay(maxDocId, numUpdates);
+  }
+  
+  /** @see #checkRandomReplay */
+  @Test
+  public void testReplay_Random_FewDocsManyUpdates() throws Exception {
+    
+    // build up a random list of updates
+    final int maxDocId = atLeast(3);
+    final int numUpdates = maxDocId * 50;
+    checkRandomReplay(maxDocId, numUpdates);
+  }
+  
+  /** @see #checkRandomReplay */
+  @Test
+  public void testReplay_Random_FewDocsManyShortSequences() throws Exception {
+    
+    // build up a random list of updates
+    final int numIters = atLeast(50);
+    
+    for (int i = 0; i < numIters; i++) {
+      final int maxDocId = atLeast(3);
+      final int numUpdates = maxDocId * 5;
+      checkRandomReplay(maxDocId, numUpdates);
+      deleteAllAndCommit();
+    }
+  }
+
+
+  /** 
+   * @see #checkReplay 
+   * @see RandomUpdate
+   */
+  public void checkRandomReplay(final int maxDocId, final int numCmds) throws Exception {
+    
+    final String not_inplaceField = "regular_l";
+    final String inplaceField = "inplace_l_dvo"; 
+
+    final Object[] cmds = new Object[numCmds];
+    for (int iter = 0; iter < numCmds; iter++) {
+      final int id = TestUtil.nextInt(random(), 1, maxDocId);
+      final RandomUpdate update = RandomUpdate.pick(random());
+
+      switch (update) {
+        
+      case HARD_COMMIT:
+        cmds[iter] = HARDCOMMIT;
+        break;
+        
+      case SOFT_COMMIT:
+        cmds[iter] = SOFTCOMMIT;
+        break;
+
+      case ATOMIC_NOT_INPLACE:
+        // atomic update on non_inplaceField, w/o any value specified for inplaceField
+        cmds[iter] = sdoc("id", id,
+                          not_inplaceField, map("inc", random().nextInt()));
+        break;
+        
+      case ATOMIC_INPLACE_AND_NOT_INPLACE:
+        // atomic update of a doc w/ inc on both inplaceField and not_inplaceField
+        cmds[iter] = sdoc("id", id,
+                          inplaceField, map("inc", random().nextInt()),
+                          not_inplaceField, map("inc", random().nextInt()));
+        break;
+
+      case ATOMIC_INPLACE_SET:
+        // atomic update of a doc w/ set inplaceField
+        cmds[iter] = sdoc("id", id,
+                          inplaceField, map("set", random().nextLong()));
+        break;
+
+      case ATOMIC_INPLACE_INC:
+        // atomic update of a doc w/ inc inplaceField
+        cmds[iter] = sdoc("id", id,
+                          inplaceField, map("inc", random().nextInt()));
+        break;
+
+      case ADD_NO_INPLACE_VALUE:
+        // regular add of doc w/o the inplaceField, but does include non_inplaceField
+        cmds[iter] = sdoc("id", id,
+                          not_inplaceField, random().nextLong());
+        break;
+
+      case ADD_INPLACE_VALUE:
+        // a non atomic update of a doc w/ new inplaceField value
+        cmds[iter] = sdoc("id", id,
+                          inplaceField, random().nextLong(),
+                          not_inplaceField, random().nextLong());
+        break;
+        
+      default:
+        fail("WTF is this? ... " + update);
+      }
+      
+      assertNotNull(cmds[iter]); // sanity check switch
+    }
+
+    checkReplay(inplaceField, cmds);
+  }
+  
+  /** sentinal object for {@link #checkReplay} */
+  public Object SOFTCOMMIT = new Object() { public String toString() { return "SOFTCOMMIT"; } };
+  /** sentinal object for {@link #checkReplay} */
+  public Object HARDCOMMIT = new Object() { public String toString() { return "HARDCOMMIT"; } };
+
+  /**
+   * Executes a sequence of commands against Solr, while tracking the expected value of a specified 
+   * <code>valField</code> Long field (presumably that only uses docvalues) against an in memory model 
+   * maintained in parallel (for the purpose of testing the correctness of in-place updates..
+   *
+   * <p>
+   * A few restrictions are placed on the {@link SolrInputDocument}s that can be included when using 
+   * this method, in order to keep the in-memory model management simple:
+   * </p>
+   * <ul>
+   *  <li><code>id</code> must be uniqueKey field</li>
+   *  <li><code>id</code> may have any FieldType, but all values must be parsable as Integers</li>
+   *  <li><code>valField</code> must be a single valued field</li>
+   *  <li>All values in the <code>valField</code> must either be {@link Number}s, or Maps containing 
+   *      atomic updates ("inc" or "set") where the atomic value is a {@link Number}</li>
+   * </ul>
+   * 
+   * @param valField the field to model
+   * @param commands A sequence of Commands which can either be SolrInputDocuments 
+   *                 (regular or containing atomic update Maps)
+   *                 or one of the {@link TestInPlaceUpdatesStandalone#HARDCOMMIT} or {@link TestInPlaceUpdatesStandalone#SOFTCOMMIT} sentinal objects.
+   */
+  public void checkReplay(final String valField, Object... commands) throws Exception {
+    
+    HashMap<Integer, DocInfo> model = new LinkedHashMap<>();
+    HashMap<Integer, DocInfo> committedModel = new LinkedHashMap<>();
+
+    // by default, we only check the committed model after a commit
+    // of if the number of total commands is relatively small.
+    //
+    // (in theory, there's no reason to check the committed model unless we know there's been a commit
+    // but for smaller tests the overhead of doing so is tiny, so we might as well)
+    //
+    // if some test seed fails, and you want to force the committed model to be checked
+    // after every command, just temporaribly force this variable to true...
+    boolean checkCommittedModel = (commands.length < 50);
+    
+    for (Object cmd : commands) {
+      if (cmd == SOFTCOMMIT) {
+        assertU(commit("softCommit", "true"));
+        committedModel = new LinkedHashMap(model);
+        checkCommittedModel = true;
+      } else if (cmd == HARDCOMMIT) {
+        assertU(commit("softCommit", "false"));
+        committedModel = new LinkedHashMap(model);
+        checkCommittedModel = true;
+      } else {
+        assertNotNull("null command in checkReplay", cmd);
+        assertTrue("cmd is neither sentinal (HARD|SOFT)COMMIT object, nor Solr doc: " + cmd.getClass(),
+                   cmd instanceof SolrInputDocument);
+        
+        final SolrInputDocument sdoc = (SolrInputDocument) cmd;
+        final int id = Integer.parseInt(sdoc.getFieldValue("id").toString());
+        
+        final DocInfo previousInfo = model.get(id);
+        final Long previousValue = (null == previousInfo) ? null : previousInfo.value;
+        
+        final long version = addAndGetVersion(sdoc, null);
+        
+        final Object val = sdoc.getFieldValue(valField);
+        if (val instanceof Map) {
+          // atomic update of the field we're modeling
+          
+          Map<String,?> atomicUpdate = (Map) val;
+          assertEquals(sdoc.toString(), 1, atomicUpdate.size());
+          if (atomicUpdate.containsKey("inc")) {
+            // Solr treats inc on a non-existing doc (or doc w/o existing value) as if existing value is 0
+            final long base = (null == previousValue) ? 0L : previousValue;
+            model.put(id, new DocInfo(version,
+                                      base + ((Number)atomicUpdate.get("inc")).longValue()));
+          } else if (atomicUpdate.containsKey("set")) {
+            model.put(id, new DocInfo(version, ((Number)atomicUpdate.get("set")).longValue()));
+          } else {
+            fail("wtf update is this? ... " + sdoc);
+          }
+        } else if (null == val) {
+          // the field we are modeling is not mentioned in this update, It's either...
+          //
+          // a) a regular update of some other fields (our model should have a null value)
+          // b) an atomic update of some other field (keep existing value in model)
+          //
+          // for now, assume it's atomic and we're going to keep our existing value...
+          Long newValue = (null == previousInfo) ? null : previousInfo.value;
+          for (SolrInputField field : sdoc) {
+            if (! ( "id".equals(field.getName()) || (field.getValue() instanceof Map)) ) {
+              // not an atomic update, newValue in model should be null
+              newValue = null;
+              break;
+            }
+          }
+          model.put(id, new DocInfo(version, newValue));
+          
+        } else {
+          // regular replacement of the value in the field we're modeling
+          
+          assertTrue("Model field value is not a Number: " + val.getClass(), val instanceof Number);
+          model.put(id, new DocInfo(version, ((Number)val).longValue()));
+        }
+      }
+
+      // after every op, check the model(s)
+      
+      // RTG to check the values for every id against the model
+      for (Map.Entry<Integer, DocInfo> entry : model.entrySet()) {
+        final Long expected = entry.getValue().value;
+        assertEquals(expected, client.getById(String.valueOf(entry.getKey())).getFirstValue(valField));
+      }
+
+      // search to check the values for every id in the committed model
+      if (checkCommittedModel) {
+        final int numCommitedDocs = committedModel.size();
+        String[] xpaths = new String[1 + numCommitedDocs];
+        int i = 0;
+        for (Map.Entry<Integer, DocInfo> entry : committedModel.entrySet()) {
+          Integer id = entry.getKey();
+          Long expected = entry.getValue().value;
+          if (null != expected) {
+            xpaths[i] = "//result/doc[./str='"+id+"'][./long='"+expected+"']";
+          } else {
+            xpaths[i] = "//result/doc[./str='"+id+"'][not(./long)]";
+          }           
+          i++;
+        }
+        xpaths[i] = "//*[@numFound='"+numCommitedDocs+"']";
+        assertQ(req("q", "*:*",
+                    "fl", "id," + valField,
+                    "rows", ""+numCommitedDocs),
+                xpaths);
+      }
+    }
+  }
+
+  @Test
+  public void testMixedInPlaceAndNonInPlaceAtomicUpdates() throws Exception {
+    SolrDocument rtgDoc = null;
+    long version1 = addAndGetVersion(sdoc("id", "1", "inplace_updatable_float", "100", "stored_i", "100"), params());
+
+    version1 = addAndAssertVersion(version1, "id", "1", "inplace_updatable_float", map("inc", "1"), "stored_i", map("inc", "1"));
+    rtgDoc = client.getById("1");
+    assertEquals(101, rtgDoc.getFieldValue("stored_i"));
+    assertEquals(101.0f, rtgDoc.getFieldValue("inplace_updatable_float"));
+    
+    version1 = addAndAssertVersion(version1, "id", "1", "inplace_updatable_float", map("inc", "1"));
+    rtgDoc = client.getById("1");
+    assertEquals(101, rtgDoc.getFieldValue("stored_i"));
+    assertEquals(102.0f, rtgDoc.getFieldValue("inplace_updatable_float"));
+
+    version1 = addAndAssertVersion(version1, "id", "1", "stored_i", map("inc", "1"));
+    rtgDoc = client.getById("1");
+    assertEquals(102, rtgDoc.getFieldValue("stored_i"));
+    assertEquals(102.0f, rtgDoc.getFieldValue("inplace_updatable_float"));
+
+    assertU(commit("softCommit", "false"));
+    assertQ(req("q", "*:*", "sort", "id asc", "fl", "*"),
+            "//*[@numFound='1']",
+            "//result/doc[1]/float[@name='inplace_updatable_float'][.='102.0']",
+            "//result/doc[1]/int[@name='stored_i'][.='102']",
+            "//result/doc[1]/long[@name='_version_'][.='" + version1 + "']"
+            );
+
+    // recheck RTG after commit
+    rtgDoc = client.getById("1");
+    assertEquals(102, rtgDoc.getFieldValue("stored_i"));
+    assertEquals(102.0f, rtgDoc.getFieldValue("inplace_updatable_float"));
+  }
+
+  /** 
+   * @see #callComputeInPlaceUpdatableFields
+   * @see AtomicUpdateDocumentMerger#computeInPlaceUpdatableFields 
+   */
+  @Test
+  public void testComputeInPlaceUpdatableFields() throws Exception {
+    Set<String> inPlaceUpdatedFields = new HashSet<String>();
+
+    // these asserts should hold true regardless of type, or wether the field has a default
+    List<String> fieldsToCheck = Arrays.asList("inplace_updatable_float",
+                                               "inplace_updatable_int",
+                                               "inplace_updatable_float_with_default",
+                                               "inplace_updatable_int_with_default");
+    Collections.shuffle(fieldsToCheck, random()); // ... and regardless of order checked
+    for (String field : fieldsToCheck) {
+      // In-place updatable field updated before it exists SHOULD NOT BE in-place updated:
+      inPlaceUpdatedFields = callComputeInPlaceUpdatableFields(sdoc("id", "1", "_version_", 42L,
+                                                                    field, map("set", 10)));
+      assertFalse(field, inPlaceUpdatedFields.contains(field));
+      
+      // In-place updatable field updated after it exists SHOULD BE in-place updated:
+      addAndGetVersion(sdoc("id", "1", field, "0"), params()); // setting up the dv
+      inPlaceUpdatedFields = callComputeInPlaceUpdatableFields(sdoc("id", "1", "_version_", 42L,
+                                                                    field, map("set", 10)));
+      assertTrue(field, inPlaceUpdatedFields.contains(field));
+
+      inPlaceUpdatedFields = callComputeInPlaceUpdatableFields(sdoc("id", "1", "_version_", 42L,
+                                                                    field, map("inc", 10)));
+      assertTrue(field, inPlaceUpdatedFields.contains(field));
+
+      final String altFieldWithDefault = field.contains("float") ?
+        "inplace_updatable_int_with_default" : "inplace_updatable_int_with_default";
+      
+      // Updating an in-place updatable field (with a default) for the first time.
+      // DV for it should have been already created when first document was indexed (above),
+      // since it has a default value
+      inPlaceUpdatedFields = callComputeInPlaceUpdatableFields(sdoc("id", "1", "_version_", 42L,
+                                                                    altFieldWithDefault, map("set", 10)));
+      assertTrue(field + " -> " + altFieldWithDefault, inPlaceUpdatedFields.contains(altFieldWithDefault));
+      
+      deleteAllAndCommit();
+    }
+  
+    // Non in-place updates
+    addAndGetVersion(sdoc("id", "1", "stored_i", "0"), params()); // setting up the dv
+    assertTrue("stored field updated",
+               callComputeInPlaceUpdatableFields(sdoc("id", "1", "_version_", 42L,
+                                                      "stored_i", map("inc", 1))).isEmpty());
+    
+    assertTrue("full document update",
+               callComputeInPlaceUpdatableFields(sdoc("id", "1", "_version_", 42L,
+                                                      "inplace_updatable_int_with_default", "100")).isEmpty());
+  
+    assertTrue("non existent dynamic dv field updated first time",
+               callComputeInPlaceUpdatableFields(sdoc("id", "1", "_version_", 42L,
+                                                      "new_updatable_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_updatable_int_i_dvo", "0"), params()); // setting up the dv
+    if (random().nextBoolean()) {
+      assertU(commit("softCommit", "false"));
+    }
+    inPlaceUpdatedFields = callComputeInPlaceUpdatableFields(sdoc("id", "2", "_version_", 42L,
+                                                                  "new_updatable_int_i_dvo", map("set", 10)));
+    assertTrue(inPlaceUpdatedFields.contains("new_updatable_int_i_dvo"));
+
+    // for copy fields, regardless of wether the source & target support inplace updates,
+    // it won't be inplace if the DVs don't exist yet...
+    assertTrue("inplace fields should be empty when doc has no copyfield src values yet",
+               callComputeInPlaceUpdatableFields(sdoc("id", "1", "_version_", 42L,
+                                                      "copyfield1_src__both_updatable", map("set", 1),
+                                                      "copyfield2_src__only_src_updatable", map("set", 2))).isEmpty());
+
+    // now add a doc that *does* have the src field for each copyfield...
+    addAndGetVersion(sdoc("id", "3",
+                          "copyfield1_src__both_updatable", -13,
+                          "copyfield2_src__only_src_updatable", -15), params()); 
+    if (random().nextBoolean()) {
+      assertU(commit("softCommit", "false"));
+    }
+    
+    // If a supported dv field has a copyField target which is supported, it should be an in-place update
+    inPlaceUpdatedFields = callComputeInPlaceUpdatableFields(sdoc("id", "3", "_version_", 42L,
+                                                                  "copyfield1_src__both_updatable", map("set", 10)));
+    assertTrue(inPlaceUpdatedFields.contains("copyfield1_src__both_updatable"));
+
+    // If a supported dv field has a copyField target which is not supported, it should not be an in-place update
+    inPlaceUpdatedFields = callComputeInPlaceUpdatableFields(sdoc("id", "3", "_version_", 42L,
+                                                                  "copyfield2_src__only_src_updatable", map("set", 10)));
+    assertTrue(inPlaceUpdatedFields.isEmpty());
+  }
+
+  @Test
+  /**
+   *  Test the @see {@link AtomicUpdateDocumentMerger#doInPlaceUpdateMerge(AddUpdateCommand,Set<String>)} 
+   *  method is working fine
+   */
+  public void testDoInPlaceUpdateMerge() throws Exception {
+    long version1 = addAndGetVersion(sdoc("id", "1", "title_s", "first"), null);
+    long version2 = addAndGetVersion(sdoc("id", "2", "title_s", "second"), null);
+    long version3 = addAndGetVersion(sdoc("id", "3", "title_s", "third"), null);
+    assertU(commit("softCommit", "false"));
+    assertQ(req("q", "*:*"), "//*[@numFound='3']");
+
+    // Adding a few in-place updates
+    version1 = addAndAssertVersion(version1, "id", "1", "inplace_updatable_float", map("set", 200));
+
+    // 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.computeInPlaceUpdatableFields(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"));
+
+    // 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.computeInPlaceUpdatableFields(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#computeInPlaceUpdatableFields} 
+   * on the specified solr input document.
+   */
+  private static Set<String> callComputeInPlaceUpdatableFields(final SolrInputDocument sdoc) throws Exception {
+    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.computeInPlaceUpdatableFields(cmd);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/53754108/solr/core/src/test/org/apache/solr/update/TestUpdate.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/TestUpdate.java b/solr/core/src/test/org/apache/solr/update/TestUpdate.java
index 6efa5bb..f16c384 100644
--- a/solr/core/src/test/org/apache/solr/update/TestUpdate.java
+++ b/solr/core/src/test/org/apache/solr/update/TestUpdate.java
@@ -32,7 +32,7 @@ public class TestUpdate extends SolrTestCaseJ4 {
   }
 
   @Test
-  public void testUpdateableDocs() throws Exception {
+  public void testUpdatableDocs() throws Exception {
     // The document may be retrieved from the index or from the transaction log.
     // Test both by running the same test with and without commits
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/53754108/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
new file mode 100644
index 0000000..9b1d611
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/update/UpdateLogTest.java
@@ -0,0 +1,271 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.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.SolrQueryRequest;
+import org.apache.solr.update.processor.DistributedUpdateProcessor;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+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
+  public static void beforeClass() throws Exception {
+    System.setProperty("solr.tests.intClassName", random().nextBoolean()? "TrieIntField": "IntPointField");
+    System.setProperty("solr.tests.longClassName", random().nextBoolean()? "TrieLongField": "LongPointField");
+    System.setProperty("solr.tests.floatClassName", random().nextBoolean()? "TrieFloatField": "FloatPointField");
+    System.setProperty("solr.tests.doubleClassName", random().nextBoolean()? "TrieDoubleField": "DoublePointField");
+
+    initCore("solrconfig-tlog.xml", "schema-inplace-updates.xml");
+
+    try (SolrQueryRequest req = req()) {
+      UpdateHandler uhandler = req.getCore().getUpdateHandler();
+      ((DirectUpdateHandler2) uhandler).getCommitTracker().setTimeUpperBound(100);
+      ((DirectUpdateHandler2) uhandler).getCommitTracker().setOpenSearcher(false);
+      ulog = uhandler.getUpdateLog();
+    }
+  }
+
+  @After
+  public void after() {
+    System.clearProperty("solr.tests.intClassName");
+    System.clearProperty("solr.tests.longClassName");
+    System.clearProperty("solr.tests.floatClassName");
+    System.clearProperty("solr.tests.doubleClassName");
+  }
+
+  @Test
+  /**
+   * @see org.apache.solr.update.UpdateLog#applyPartialUpdates(BytesRef,long,long,SolrDocumentBase)
+   */
+  public void testApplyPartialUpdatesOnMultipleInPlaceUpdatesInSequence() {    
+    // Add a full update, two in-place updates and verify applying partial updates is working
+    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(DOC_1_INDEXED_ID);
+    SolrDocument partialDoc = RealTimeGetComponent.toSolrDoc((SolrInputDocument)((List)partialUpdate).get(4), 
+        h.getCore().getLatestSchema());
+    long prevVersion = (Long)((List)partialUpdate).get(3);
+    long prevPointer = (Long)((List)partialUpdate).get(2);
+
+    assertEquals(3L, ((NumericDocValuesField)partialDoc.getFieldValue("val1_i_dvo")).numericValue());
+    assertFalse(partialDoc.containsKey("title_s"));
+
+    long returnVal = ulog.applyPartialUpdates(DOC_1_INDEXED_ID, prevPointer, prevVersion, null, partialDoc);
+
+    assertEquals(0, returnVal);
+    assertEquals(1000, Integer.parseInt(partialDoc.getFieldValue("price").toString()));
+    assertEquals(3L, ((NumericDocValuesField)partialDoc.getFieldValue("val1_i_dvo")).numericValue());
+    assertEquals("title1", partialDoc.getFieldValue("title_s"));
+
+    // Add a full update, commit, then two in-place updates, and verify that applying partial updates is working (since
+    // the prevTlog and prevTlog2 are retained after a commit
+    ulogCommit(ulog);
+    if (random().nextBoolean()) { // sometimes also try a second commit
+      ulogCommit(ulog);
+    }
+    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(DOC_1_INDEXED_ID);
+    partialDoc = RealTimeGetComponent.toSolrDoc((SolrInputDocument)((List)partialUpdate).get(4), h.getCore().getLatestSchema());
+    prevVersion = (Long)((List)partialUpdate).get(3);
+    prevPointer = (Long)((List)partialUpdate).get(2);
+
+    assertEquals(5L, ((NumericDocValuesField)partialDoc.getFieldValue("val1_i_dvo")).numericValue());
+    assertFalse(partialDoc.containsKey("title_s"));
+
+    returnVal = ulog.applyPartialUpdates(DOC_1_INDEXED_ID, prevPointer, prevVersion, null, partialDoc);
+
+    assertEquals(0, returnVal);
+    assertEquals(2000, Integer.parseInt(partialDoc.getFieldValue("price").toString()));
+    assertEquals(5L, ((NumericDocValuesField)partialDoc.getFieldValue("val1_i_dvo")).numericValue());
+    assertEquals("title1", partialDoc.getFieldValue("title_s"));
+  }
+  
+  @Test
+  public void testApplyPartialUpdatesAfterMultipleCommits() {    
+    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);
+    ulogAdd(ulog, 101L, sdoc("id", "1", "val1_i_dvo", "6", "_version_", "300"));
+
+    Object partialUpdate = ulog.lookup(DOC_1_INDEXED_ID);
+    SolrDocument partialDoc = RealTimeGetComponent.toSolrDoc((SolrInputDocument)((List)partialUpdate).get(4), h.getCore().getLatestSchema());
+    long prevVersion = (Long)((List)partialUpdate).get(3);
+    long prevPointer = (Long)((List)partialUpdate).get(2);
+
+    assertEquals(6L, ((NumericDocValuesField)partialDoc.getFieldValue("val1_i_dvo")).numericValue());
+    assertFalse(partialDoc.containsKey("title_s"));
+
+    long returnVal = ulog.applyPartialUpdates(DOC_1_INDEXED_ID, prevPointer, prevVersion, null, partialDoc);
+
+    assertEquals(-1, returnVal);
+  }
+
+  @Test
+  public void testApplyPartialUpdatesDependingOnNonAddShouldThrowException() {
+    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"));
+
+    Object partialUpdate = ulog.lookup(DOC_1_INDEXED_ID);
+    SolrDocument partialDoc = RealTimeGetComponent.toSolrDoc((SolrInputDocument)((List)partialUpdate).get(4), h.getCore().getLatestSchema());
+    long prevVersion = (Long)((List)partialUpdate).get(3);
+    long prevPointer = (Long)((List)partialUpdate).get(2);
+
+    assertEquals(3L, ((NumericDocValuesField)partialDoc.getFieldValue("val1_i_dvo")).numericValue());
+    assertEquals(502L, ((NumericDocValuesField)partialDoc.getFieldValue("_version_")).numericValue());
+    assertFalse(partialDoc.containsKey("title_s"));
+
+    // 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(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);
+      });
+    assertEquals(ex.toString(), SolrException.ErrorCode.INVALID_STATE.code, ex.code());
+    assertThat(ex.getMessage(), containsString("should've been either ADD or UPDATE_INPLACE"));
+    assertThat(ex.getMessage(), containsString("looking for id=1"));
+  }
+
+  @Test
+  public void testApplyPartialUpdatesWithDelete() throws Exception {
+    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")); // in-place update
+    ulogAdd(ulog, 101L, sdoc("id", "1", "val1_i_dvo", "3", "_version_", "102")); // in-place update
+    
+    // sanity check that the update log has one document, and RTG returns the document
+    assertEquals(1, ulog.map.size());
+    assertJQ(req("qt","/get", "id","1")
+             , "=={'doc':{ 'id':'1', 'val1_i_dvo':3, '_version_':102, 'title_s':'title1', "
+             // fields with default values
+             + "'inplace_updatable_int_with_default':666, 'inplace_updatable_float_with_default':42.0}}");
+    
+    boolean dbq = random().nextBoolean();
+    ulogDelete(ulog, "1", 200L, dbq); // delete id:1 document
+    if (dbq) {
+      assertNull(ulog.lookup(DOC_1_INDEXED_ID)); // any DBQ clears out the ulog, so this document shouldn't exist
+      assertEquals(0, ulog.map.size());
+      assertTrue(String.valueOf(ulog.prevMap), ulog.prevMap == null || ulog.prevMap.size() == 0);
+      assertTrue(String.valueOf(ulog.prevMap2), ulog.prevMap2 == null || ulog.prevMap2.size() == 0);
+      // verify that the document is deleted, by doing an RTG call
+      assertJQ(req("qt","/get", "id","1"), "=={'doc':null}");
+    } else { // dbi
+      List entry = ((List)ulog.lookup(DOC_1_INDEXED_ID));
+      assertEquals(UpdateLog.DELETE, (int)entry.get(UpdateLog.FLAGS_IDX) & UpdateLog.OPERATION_MASK);
+    }
+  }
+
+  /**
+   * Simulate a commit on a given updateLog
+   */
+  private static void ulogCommit(UpdateLog ulog) {
+    try (SolrQueryRequest req = req()) {
+      CommitUpdateCommand commitCmd = new CommitUpdateCommand(req, false);
+      ulog.preCommit(commitCmd);
+      ulog.postCommit(commitCmd);
+    }
+  }
+
+  /**
+   * 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 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);
+      }
+    }
+  }
+
+  /**
+   * 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
+   */
+  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));
+    cmd.setVersion(Long.parseLong(cmd.solrDoc.getFieldValue(DistributedUpdateProcessor.VERSION_FIELD).toString()));
+    return cmd;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/53754108/solr/core/src/test/org/apache/solr/update/processor/AtomicUpdatesTest.java
----------------------------------------------------------------------
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 a91a34b..2d07af7 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
@@ -17,6 +17,7 @@
 package org.apache.solr.update.processor;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Date;
 import java.util.List;
 
@@ -1135,4 +1136,150 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
     assertQ(req("q", "cat:ccc", "indent", "true"), "//result[@numFound = '1']");
 
   }
+
+  public void testFieldsWithDefaultValuesWhenAtomicUpdatesAgainstTlog() {
+    for (String fieldToUpdate : Arrays.asList("field_to_update_i1", "field_to_update_i_dvo")) {
+      clearIndex();
+      
+      assertU(adoc(sdoc("id", "7", fieldToUpdate, "666")));
+      assertQ(fieldToUpdate + ": initial RTG"
+              , req("qt", "/get", "id", "7")
+              , "count(//doc)=1"
+              , "//doc/int[@name='id'][.='7']"
+              , "//doc/int[@name='"+fieldToUpdate+"'][.='666']"
+              , "//doc/int[@name='intDefault'][.='42']"
+              , "//doc/int[@name='intDvoDefault'][.='42']"
+              , "//doc/long[@name='_version_']"
+              , "//doc/date[@name='timestamp']"
+              , "//doc/arr[@name='multiDefault']/str[.='muLti-Default']"
+              , "count(//doc/*)=7"
+              );
+
+      // do atomic update
+      assertU(adoc(sdoc("id", "7", fieldToUpdate, ImmutableMap.of("inc", -555))));
+      assertQ(fieldToUpdate + ": RTG after atomic update"
+              , req("qt", "/get", "id", "7")
+              , "count(//doc)=1"
+              , "//doc/int[@name='id'][.='7']"
+              , "//doc/int[@name='"+fieldToUpdate+"'][.='111']"
+              , "//doc/int[@name='intDefault'][.='42']"
+              , "//doc/int[@name='intDvoDefault'][.='42']"
+              , "//doc/long[@name='_version_']"
+              , "//doc/date[@name='timestamp']"
+              , "//doc/arr[@name='multiDefault']/str[.='muLti-Default']"
+              , "count(//doc/*)=7"
+              );
+
+      assertU(commit());
+      assertQ(fieldToUpdate + ": post commit RTG"
+              , req("qt", "/get", "id", "7")
+              , "count(//doc)=1"
+              , "//doc/int[@name='id'][.='7']"
+              , "//doc/int[@name='"+fieldToUpdate+"'][.='111']"
+              , "//doc/int[@name='intDefault'][.='42']"
+              , "//doc/int[@name='intDvoDefault'][.='42']"
+              , "//doc/long[@name='_version_']"
+              , "//doc/date[@name='timestamp']"
+              , "//doc/arr[@name='multiDefault']/str[.='muLti-Default']"
+              , "count(//doc/*)=7"
+              );
+    }
+    
+  }
+
+  @AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-9838")
+  public void testAtomicUpdateOfFieldsWithDefaultValue() {
+    // both fields have the same default value (42)
+    for (String fieldToUpdate : Arrays.asList("intDefault", "intDvoDefault")) {
+      clearIndex();
+
+      // doc where we immediately attempt to inc the default value
+      assertU(adoc(sdoc("id", "7", fieldToUpdate, ImmutableMap.of("inc", "666"))));
+      assertQ(fieldToUpdate + ": initial RTG#7"
+              , req("qt", "/get", "id", "7")
+              , "count(//doc)=1"
+              , "//doc/int[@name='id'][.='7']"
+              , "//doc/int[@name='"+fieldToUpdate+"'][.='708']"
+              // whichever field we did *NOT* update
+              , "//doc/int[@name!='"+fieldToUpdate+"'][.='42']"
+              , "//doc/long[@name='_version_']"
+              , "//doc/date[@name='timestamp']"
+              , "//doc/arr[@name='multiDefault']/str[.='muLti-Default']"
+              , "count(//doc/*)=6"
+              );
+      // do atomic update
+      assertU(adoc(sdoc("id", "7", fieldToUpdate, ImmutableMap.of("inc", -555))));
+      assertQ(fieldToUpdate + ": RTG#7 after atomic update"
+              , req("qt", "/get", "id", "7")
+              , "count(//doc)=1"
+              , "//doc/int[@name='id'][.='7']"
+              , "//doc/int[@name='"+fieldToUpdate+"'][.='153']"
+              // whichever field we did *NOT* update
+              , "//doc/int[@name!='"+fieldToUpdate+"'][.='42']"
+              , "//doc/long[@name='_version_']"
+              , "//doc/date[@name='timestamp']"
+              , "//doc/arr[@name='multiDefault']/str[.='muLti-Default']"
+              , "count(//doc/*)=6"
+              );
+
+      // diff doc where we check that we can overwrite the default value
+      assertU(adoc(sdoc("id", "8", fieldToUpdate, ImmutableMap.of("set", "666"))));
+      assertQ(fieldToUpdate + ": initial RTG#8"
+              , req("qt", "/get", "id", "8")
+              , "count(//doc)=1"
+              , "//doc/int[@name='id'][.='8']"
+              , "//doc/int[@name='"+fieldToUpdate+"'][.='666']"
+              // whichever field we did *NOT* update
+              , "//doc/int[@name!='"+fieldToUpdate+"'][.='42']"
+              , "//doc/long[@name='_version_']"
+              , "//doc/date[@name='timestamp']"
+              , "//doc/arr[@name='multiDefault']/str[.='muLti-Default']"
+              , "count(//doc/*)=6"
+              );
+      // do atomic update
+      assertU(adoc(sdoc("id", "7", fieldToUpdate, ImmutableMap.of("inc", -555))));
+      assertQ(fieldToUpdate + ": RTG after atomic update"
+              , req("qt", "/get", "id", "8")
+              , "count(//doc)=1"
+              , "//doc/int[@name='id'][.='8']"
+              , "//doc/int[@name='"+fieldToUpdate+"'][.='111']"
+              // whichever field we did *NOT* update
+              , "//doc/int[@name!='"+fieldToUpdate+"'][.='42']"
+              , "//doc/long[@name='_version_']"
+              , "//doc/date[@name='timestamp']"
+              , "//doc/arr[@name='multiDefault']/str[.='muLti-Default']"
+              , "count(//doc/*)=6"
+              );
+      
+      assertU(commit());
+      
+      assertQ(fieldToUpdate + ": doc7 post commit RTG"
+              , req("qt", "/get", "id", "7")
+              , "count(//doc)=1"
+              , "//doc/int[@name='id'][.='7']"
+              , "//doc/int[@name='"+fieldToUpdate+"'][.='153']"
+              // whichever field we did *NOT* update
+              , "//doc/int[@name!='"+fieldToUpdate+"'][.='42']"
+              , "//doc/long[@name='_version_']"
+              , "//doc/date[@name='timestamp']"
+              , "//doc/arr[@name='multiDefault']/str[.='muLti-Default']"
+              , "count(//doc/*)=6"
+              );
+      assertQ(fieldToUpdate + ": doc8 post commit RTG"
+              , req("qt", "/get", "id", "8")
+              , "count(//doc)=1"
+              , "//doc/int[@name='id'][.='8']"
+              , "//doc/int[@name='"+fieldToUpdate+"'][.='111']"
+              // whichever field we did *NOT* update
+              , "//doc/int[@name!='"+fieldToUpdate+"'][.='42']"
+              , "//doc/long[@name='_version_']"
+              , "//doc/date[@name='timestamp']"
+              , "//doc/arr[@name='multiDefault']/str[.='muLti-Default']"
+              , "count(//doc/*)=6"
+              );
+    }
+    
+  }
+  
+  
 }