You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by dw...@apache.org on 2021/03/10 09:53:19 UTC

[lucene] 05/45: SOLR-15150: New update.partial.requireInPlace=true option to prevent any partial document updates that can't be done In-Place

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

dweiss pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 484a336c4fe344f96fd7370f26d56a02256c1dc2
Author: Chris Hostetter <ho...@apache.org>
AuthorDate: Tue Feb 16 15:04:56 2021 -0700

    SOLR-15150: New update.partial.requireInPlace=true option to prevent any partial document updates that can't be done In-Place
    
    (cherry picked from commit 1c7dac83075a9e214311a85c0f85dd72fe6f444a)
---
 solr/CHANGES.txt                                   |   3 +
 .../processor/DistributedUpdateProcessor.java      |  20 +++-
 .../update/TestInPlaceUpdateWithRouteField.java    |  21 ++++
 .../solr/update/TestInPlaceUpdatesStandalone.java  | 121 +++++++++++++++++++++
 .../src/updating-parts-of-documents.adoc           |  12 +-
 .../apache/solr/common/params/UpdateParams.java    |   6 +
 6 files changed, 179 insertions(+), 4 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 4b27730..dbcd18d 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -12,6 +12,9 @@ New Features
 ---------------------
 * SOLR-14330: ExpandComponent now supports an expand.nullGroup=true option (hossman)
 
+* SOLR-15150: New update.partial.requireInPlace=true option to prevent any partial document updates that can't be done In-Place (hossman)
+
+
 Improvements
 ---------------------
 * SOLR-15081: Metrics for a core: add SolrCloud "isLeader" and "replicaState".  (David Smiley)
diff --git a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
index 5b7800c..4802134 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
@@ -184,8 +184,15 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
 
     // this should always be used - see filterParams
     DistributedUpdateProcessorFactory.addParamToDistributedRequestWhitelist
-      (this.req, UpdateParams.UPDATE_CHAIN, TEST_DISTRIB_SKIP_SERVERS, CommonParams.VERSION_FIELD,
-          UpdateParams.EXPUNGE_DELETES, UpdateParams.OPTIMIZE, UpdateParams.MAX_OPTIMIZE_SEGMENTS, ShardParams._ROUTE_);
+      (this.req,
+       UpdateParams.UPDATE_CHAIN,
+       TEST_DISTRIB_SKIP_SERVERS,
+       CommonParams.VERSION_FIELD,
+       UpdateParams.EXPUNGE_DELETES,
+       UpdateParams.OPTIMIZE,
+       UpdateParams.MAX_OPTIMIZE_SEGMENTS,
+       UpdateParams.REQUIRE_PARTIAL_DOC_UPDATES_INPLACE,
+       ShardParams._ROUTE_);
 
     //this.rsp = reqInfo != null ? reqInfo.getRsp() : null;
   }
@@ -683,6 +690,15 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
       } // in-place update failed, so fall through and re-try the same with a full atomic update
     }
 
+    // if this is an atomic update, and hasn't already been done "in-place",
+    // but the user indicated it must be done in palce, then fail with an error...
+    if (cmd.getReq().getParams().getBool(UpdateParams.REQUIRE_PARTIAL_DOC_UPDATES_INPLACE, false)) {
+      throw new SolrException
+        (ErrorCode.BAD_REQUEST,
+         "Can not satisfy '" + UpdateParams.REQUIRE_PARTIAL_DOC_UPDATES_INPLACE +
+         "'; Unable to update doc in-place: " + cmd.getPrintableId());
+    }
+    
     // full (non-inplace) atomic update
 
     final SolrInputDocument oldRootDocWithChildren =
diff --git a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdateWithRouteField.java b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdateWithRouteField.java
index 1e0b6f8..0fbc463 100644
--- a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdateWithRouteField.java
+++ b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdateWithRouteField.java
@@ -42,10 +42,13 @@ 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.params.UpdateParams;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import static org.hamcrest.core.StringContains.containsString;
+
 public class TestInPlaceUpdateWithRouteField extends SolrCloudTestCase {
 
   private static final int NUMBER_OF_DOCS = 100;
@@ -107,6 +110,11 @@ public class TestInPlaceUpdateWithRouteField extends SolrCloudTestCase {
     
     UpdateRequest updateRequest = new UpdateRequest()
         .add(sdoc);
+    
+    // since this atomic update will be done in place, it shouldn't matter if we specify this param, or what it's value is
+    if (random().nextBoolean()) {
+      updateRequest.setParam(UpdateParams.REQUIRE_PARTIAL_DOC_UPDATES_INPLACE, Boolean.toString(random().nextBoolean()));
+    }
     updateRequest.commit(cluster.getSolrClient(), COLLECTION);
     solrDocument = queryDoc(id);
     Long newVersion = (Long) solrDocument.get("_version_");
@@ -120,6 +128,19 @@ public class TestInPlaceUpdateWithRouteField extends SolrCloudTestCase {
 
     sdoc.addField("shardName",  map("set", "newShardName"));
     checkWrongCommandFailure(sdoc);
+
+    sdoc.setField("shardName", shardName);
+    
+    // if we now attempt an atomic update that we know can't be done in-place, this should fail...
+    sdoc.addField("title_s", map("set", "this is a string that can't be updated in place"));
+    final SolrException e = expectThrows(SolrException.class, () -> {
+        final UpdateRequest r = new UpdateRequest();
+        r.add(sdoc);
+        r.setParam(UpdateParams.REQUIRE_PARTIAL_DOC_UPDATES_INPLACE, "true");
+        r.process(cluster.getSolrClient(), COLLECTION);
+      });
+    assertEquals(SolrException.ErrorCode.BAD_REQUEST.code, e.code());
+    assertThat(e.getMessage(), containsString("Unable to update doc in-place: " + id));
   }
 
   private void checkWrongCommandFailure(SolrInputDocument sdoc) throws SolrServerException, IOException {
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 91586ca..f61ad6c 100644
--- a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java
+++ b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java
@@ -43,6 +43,8 @@ import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.SolrInputField;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.params.UpdateParams;
 import org.apache.solr.index.NoMergePolicyFactory;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.schema.IndexSchema;
@@ -151,6 +153,13 @@ public class TestInPlaceUpdatesStandalone extends SolrTestCaseJ4 {
             map("inc", new ArrayList<>(Collections.singletonList(123)))));
     assertEquals(SolrException.ErrorCode.BAD_REQUEST.code, e.code());
     MatcherAssert.assertThat(e.getMessage(), containsString("Invalid input '[123]' for field inplace_updatable_float"));
+
+    // regular atomic update should fail if user says they only want in-place atomic updates...
+    e = expectThrows(SolrException.class,
+                     () -> addAndGetVersion(sdoc("id", "1", "regular_l", map("inc", 1)),
+                                            params(UpdateParams.REQUIRE_PARTIAL_DOC_UPDATES_INPLACE, "true")));
+    assertEquals(SolrException.ErrorCode.BAD_REQUEST.code, e.code());
+    MatcherAssert.assertThat(e.getMessage(), containsString("Unable to update doc in-place: 1"));
   }
 
   @Test
@@ -321,6 +330,118 @@ public class TestInPlaceUpdatesStandalone extends SolrTestCaseJ4 {
         "//result/doc[2]/float[@name='inplace_updatable_float'][.='102.0']");
   }
 
+  public void testUserRequestedFailIfNotInPlace() throws Exception {
+    final SolrParams require_inplace = params(UpdateParams.REQUIRE_PARTIAL_DOC_UPDATES_INPLACE, "true");
+    long v;
+    
+    // regular updates should be ok even if require_inplace params are used,
+    // that way true "adds" wil work even if require_inplace params are in in "/update" defaults or invariants...
+    long version1 = addAndGetVersion(sdoc("id", "1", "title_s", "first", "regular_l", 1, "inplace_updatable_float", 41), require_inplace);
+    long version2 = addAndGetVersion(sdoc("id", "2", "title_s", "second", "regular_l", 2, "inplace_updatable_float", 42), require_inplace);
+    long version3 = addAndGetVersion(sdoc("id", "3", "title_s", "third", "regular_l", 3, "inplace_updatable_float", 43), require_inplace);
+    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
+    final int docid1 = getDocId("1");
+    final int docid2 = getDocId("2");
+    final int docid3 = getDocId("3");
+
+    // this atomic update should be done in place...
+    v = addAndGetVersion(sdoc("id", "2", "inplace_updatable_float", map("inc", 2)), require_inplace);
+    assertTrue(v > version2);
+    version2 = v;
+
+    // this atomic update should also be done in place, even though the user didn't insist on it...
+    v = addAndGetVersion(sdoc("id", "3", "inplace_updatable_float", map("inc", 3)), params());
+    assertTrue(v > version3);
+    version3 = v;
+
+    assertU(commit("softCommit", "false"));
+    assertQ(req("q", "*:*", "sort", "id asc", "fl", "*,[docid]")
+            , "//*[@numFound='3']"
+            , "//result/doc[1]/long[@name='regular_l'][.='1']"
+            , "//result/doc[2]/long[@name='regular_l'][.='2']"
+            , "//result/doc[3]/long[@name='regular_l'][.='3']"
+            , "//result/doc[1]/float[@name='inplace_updatable_float'][.='41.0']"
+            , "//result/doc[2]/float[@name='inplace_updatable_float'][.='44.0']"
+            , "//result/doc[3]/float[@name='inplace_updatable_float'][.='46.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+"']"
+            );
+
+    // this is an atomic update, but it can't be done in-place, so it should fail w/o affecting index...
+    SolrException e = expectThrows(SolrException.class,
+                                   () -> addAndGetVersion(sdoc("id", "1", "regular_l", map("inc", 1)),
+                                                          require_inplace));
+    assertEquals(SolrException.ErrorCode.BAD_REQUEST.code, e.code());
+    MatcherAssert.assertThat(e.getMessage(), containsString("Unable to update doc in-place: 1"));
+
+    // data in solr should be unchanged after failed attempt at non-inplace atomic update...
+    assertU(commit("softCommit", "false"));
+    assertQ(req("q", "*:*", "sort", "id asc", "fl", "*,[docid]")
+            , "//*[@numFound='3']"
+            , "//result/doc[1]/long[@name='regular_l'][.='1']"
+            , "//result/doc[2]/long[@name='regular_l'][.='2']"
+            , "//result/doc[3]/long[@name='regular_l'][.='3']"
+            , "//result/doc[1]/float[@name='inplace_updatable_float'][.='41.0']"
+            , "//result/doc[2]/float[@name='inplace_updatable_float'][.='44.0']"
+            , "//result/doc[3]/float[@name='inplace_updatable_float'][.='46.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+"']"
+            );
+
+    
+    // the same atomic update w/o require_inplace params should proceed, and can modify the docid(s)
+    // (but we don't assert that, since it the merge policy might kick in
+    v = addAndGetVersion(sdoc("id", "1", "regular_l", map("inc", 100)), params());
+    assertTrue(v > version1);
+    version1 = v;
+
+    assertU(commit("softCommit", "false"));
+    assertQ(req("q", "*:*", "sort", "id asc", "fl", "*")
+            , "//*[@numFound='3']"
+            , "//result/doc[1]/long[@name='regular_l'][.='101']"
+            , "//result/doc[2]/long[@name='regular_l'][.='2']"
+            , "//result/doc[3]/long[@name='regular_l'][.='3']"
+            , "//result/doc[1]/float[@name='inplace_updatable_float'][.='41.0']"
+            , "//result/doc[2]/float[@name='inplace_updatable_float'][.='44.0']"
+            , "//result/doc[3]/float[@name='inplace_updatable_float'][.='46.0']"
+            , "//result/doc[1]/long[@name='_version_'][.='"+version1+"']"
+            , "//result/doc[2]/long[@name='_version_'][.='"+version2+"']"
+            , "//result/doc[3]/long[@name='_version_'][.='"+version3+"']"
+            );
+
+    // a regular old re-indexing of a document should also succeed, even w/require_inplace, since it's not ant atomic update
+    v = addAndGetVersion(sdoc("id", "1", "regular_l", "999"), require_inplace);
+    assertTrue(v > version1);
+    version1 = v;
+    
+    assertU(commit("softCommit", "false"));
+    assertQ(req("q", "*:*", "sort", "id asc", "fl", "*")
+            , "//*[@numFound='3']"
+            , "//result/doc[1]/long[@name='regular_l'][.='999']"
+            , "//result/doc[2]/long[@name='regular_l'][.='2']"
+            , "//result/doc[3]/long[@name='regular_l'][.='3']"
+            , "0=count(//result/doc[1]/float[@name='inplace_updatable_float'])" // not in new doc
+            , "//result/doc[2]/float[@name='inplace_updatable_float'][.='44.0']"
+            , "//result/doc[3]/float[@name='inplace_updatable_float'][.='46.0']"
+            , "//result/doc[1]/long[@name='_version_'][.='"+version1+"']"
+            , "//result/doc[2]/long[@name='_version_'][.='"+version2+"']"
+            , "//result/doc[3]/long[@name='_version_'][.='"+version3+"']"
+            );
+
+  }
+  
   @Test
   public void testUpdatingFieldNotPresentInDoc() throws Exception {
     long version1 = addAndGetVersion(sdoc("id", "1", "title_s", "first"), null);
diff --git a/solr/solr-ref-guide/src/updating-parts-of-documents.adoc b/solr/solr-ref-guide/src/updating-parts-of-documents.adoc
index 4203d2b..094e119 100644
--- a/solr/solr-ref-guide/src/updating-parts-of-documents.adoc
+++ b/solr/solr-ref-guide/src/updating-parts-of-documents.adoc
@@ -211,9 +211,9 @@ curl -X POST 'http://localhost:8983/solr/gettingstarted/update?commit=true' -H '
 
 == In-Place Updates
 
-In-place updates are very similar to atomic updates; in some sense, this is a subset of atomic updates. In regular atomic updates, the entire document is reindexed internally during the application of the update. However, in this approach, only the fields to be updated are affected and the rest of the documents are not reindexed internally. Hence, the efficiency of updating in-place is unaffected by the size of the documents that are updated (i.e., number of fields, size of fields, etc.) [...]
+In-place updates are very similar to atomic updates; in some sense, this is a subset of atomic updates. In regular atomic updates, the entire document is reindexed internally during the application of the update. However, in this approach, only the fields to be updated are affected and the rest of the documents are not reindexed internally. Hence, the efficiency of updating in-place is unaffected by the size of the documents that are updated (i.e., number of fields, size of fields, etc.) [...]
 
-An atomic update operation is performed using this approach only when the fields to be updated meet these three conditions:
+An atomic update operation is performed using this In-Place approach only when the fields to be updated meet these three conditions:
 
 * are non-indexed (`indexed="false"`), non-stored (`stored="false"`), single valued (`multiValued="false"`) numeric docValues (`docValues="true"`) fields;
 * the `\_version_` field is also a non-indexed, non-stored single valued docValues field; and,
@@ -227,6 +227,14 @@ Set or replace the field value(s) with the specified value(s). May be specified
 `inc`::
 Increments a numeric value by a specific amount. Must be specified as a single numeric value.
 
+[TIP]
+====
+.Preventing Atomic Updates that can't be done In-Place
+
+Since it can be tricky to ensure that all of the neccessary conditions are satisfied to ensure that an udpate can be don In-Place, Solr supports a request parameter option named `update.partial.requireInPlace`.  When set to `true`, and Atomic Update that can not be don In-Place will fail.  Users can specify this option when they would prefer that an update request "fail fast" if it can't be done In-Place.
+
+====
+
 === In-Place Update Example
 
 If the price and popularity fields are defined in the schema as:
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/UpdateParams.java b/solr/solrj/src/java/org/apache/solr/common/params/UpdateParams.java
index c4633bd..901da37 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/UpdateParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/UpdateParams.java
@@ -68,4 +68,10 @@ public interface UpdateParams
 
   /** Return versions of updates? */
   public static final String VERSIONS = "versions";
+
+  /**
+   * If set to true, then Solr must fail to process any Atomic Update which can not 
+   * be done "In-Place" with out re-indexing the entire document.
+   */
+  public static final String REQUIRE_PARTIAL_DOC_UPDATES_INPLACE = "update.partial.requireInPlace";
 }